Skip to content

Commit

Permalink
Fix issue that message ordering could be broken when redelivering mes…
Browse files Browse the repository at this point in the history
…sages on Key_Shared subscription
  • Loading branch information
Masahiro Sakamoto committed Jun 29, 2021
1 parent a48b725 commit 401f4ee
Show file tree
Hide file tree
Showing 12 changed files with 569 additions and 67 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -104,9 +104,7 @@ public void removeConsumer(Consumer consumer) {
}

@Override
public Consumer select(byte[] stickyKey) {
int hash = Murmur3_32Hash.getInstance().makeHash(stickyKey);

public Consumer select(int hash) {
rwLock.readLock().lock();
try {
if (hashRing.isEmpty()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import org.apache.pulsar.common.api.proto.MessageIdData;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.stats.Rate;
import org.apache.pulsar.common.util.DateFormatter;
import org.apache.pulsar.common.util.FutureUtil;
Expand Down Expand Up @@ -235,7 +236,8 @@ public Future<Void> sendMessages(final List<Entry> entries, EntryBatchSizes batc
Entry entry = entries.get(i);
if (entry != null) {
int batchSize = batchSizes.getBatchSize(i);
pendingAcks.put(entry.getLedgerId(), entry.getEntryId(), batchSize, 0);
int stickyKeyHash = getStickyKeyHash(entry);
pendingAcks.put(entry.getLedgerId(), entry.getEntryId(), batchSize, stickyKeyHash);
if (log.isDebugEnabled()){
log.debug("[{}-{}] Added {}:{} ledger entry with batchSize of {} to pendingAcks in"
+ " broker.service.Consumer for consumerId: {}",
Expand Down Expand Up @@ -742,7 +744,7 @@ public void redeliverUnacknowledgedMessages() {
if (pendingAcks != null) {
List<PositionImpl> pendingPositions = new ArrayList<>((int) pendingAcks.size());
MutableInt totalRedeliveryMessages = new MutableInt(0);
pendingAcks.forEach((ledgerId, entryId, batchSize, none) -> {
pendingAcks.forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> {
totalRedeliveryMessages.add((int) batchSize);
pendingPositions.add(new PositionImpl(ledgerId, entryId));
});
Expand All @@ -765,10 +767,11 @@ public void redeliverUnacknowledgedMessages(List<MessageIdData> messageIds) {
List<PositionImpl> pendingPositions = Lists.newArrayList();
for (MessageIdData msg : messageIds) {
PositionImpl position = PositionImpl.get(msg.getLedgerId(), msg.getEntryId());
LongPair batchSize = pendingAcks.get(position.getLedgerId(), position.getEntryId());
if (batchSize != null) {
LongPair longPair = pendingAcks.get(position.getLedgerId(), position.getEntryId());
if (longPair != null) {
long batchSize = longPair.first;
pendingAcks.remove(position.getLedgerId(), position.getEntryId());
totalRedeliveryMessages += batchSize.first;
totalRedeliveryMessages += batchSize;
pendingPositions.add(position);
}
}
Expand Down Expand Up @@ -835,5 +838,10 @@ public String getClientAddress() {
return clientAddress;
}

private int getStickyKeyHash(Entry entry) {
byte[] stickyKey = Commands.peekStickyKey(entry.getDataBuffer(), topicName, subscription.getName());
return StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey);
}

private static final Logger log = LoggerFactory.getLogger(Consumer.class);
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException;
import org.apache.pulsar.common.util.Murmur3_32Hash;

/**
* This is a consumer selector based fixed hash range.
Expand Down Expand Up @@ -103,8 +102,7 @@ public synchronized void removeConsumer(Consumer consumer) {
}

@Override
public Consumer select(byte[] stickyKey) {
int hash = Murmur3_32Hash.getInstance().makeHash(stickyKey);
public Consumer select(int hash) {
if (rangeMap.size() > 0) {
int slot = hash % rangeSize;
return rangeMap.ceilingEntry(slot).getValue();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.pulsar.common.api.proto.IntRange;
import org.apache.pulsar.common.api.proto.KeySharedMeta;
import org.apache.pulsar.common.util.Murmur3_32Hash;

/**
* This is a sticky-key consumer selector based user provided range.
Expand Down Expand Up @@ -65,11 +64,6 @@ public void removeConsumer(Consumer consumer) {
rangeMap.entrySet().removeIf(entry -> entry.getValue().equals(consumer));
}

@Override
public Consumer select(byte[] stickyKey) {
return select(Murmur3_32Hash.getInstance().makeHash(stickyKey));
}

@Override
public Map<String, List<String>> getConsumerKeyHashRanges() {
Map<String, List<String>> result = new HashMap<>();
Expand All @@ -88,7 +82,8 @@ public Map<String, List<String>> getConsumerKeyHashRanges() {
return result;
}

Consumer select(int hash) {
@Override
public Consumer select(int hash) {
if (rangeMap.size() > 0) {
int slot = hash % rangeSize;
Map.Entry<Integer, Consumer> ceilingEntry = rangeMap.ceilingEntry(slot);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.util.List;
import java.util.Map;
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException;
import org.apache.pulsar.common.util.Murmur3_32Hash;

public interface StickyKeyConsumerSelector {

Expand All @@ -45,7 +46,21 @@ public interface StickyKeyConsumerSelector {
* @param stickyKey sticky key
* @return consumer
*/
Consumer select(byte[] stickyKey);
default Consumer select(byte[] stickyKey) {
return select(makeStickyKeyHash(stickyKey));
}

static int makeStickyKeyHash(byte[] stickyKey) {
return Murmur3_32Hash.getInstance().makeHash(stickyKey);
}

/**
* Select a consumer by hash.
*
* @param hash hash corresponding to sticky key
* @return consumer
*/
Consumer select(int hash);

/**
* Get key hash ranges handled by each consumer.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.service.persistent;

import com.google.common.collect.ComparisonChain;
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap.LongPair;
import org.apache.pulsar.common.util.collections.ConcurrentSortedLongPairSet;
import org.apache.pulsar.common.util.collections.LongPairSet;

public class MessageRedeliveryController {
private final LongPairSet messagesToRedeliver;
private final ConcurrentLongLongPairHashMap hashesToBeBlocked;

public MessageRedeliveryController(boolean allowOutOfOrderDelivery) {
this.messagesToRedeliver = new ConcurrentSortedLongPairSet(128, 2);
this.hashesToBeBlocked = allowOutOfOrderDelivery ? null : new ConcurrentLongLongPairHashMap(128, 2);
}

public boolean add(long ledgerId, long entryId) {
return messagesToRedeliver.add(ledgerId, entryId);
}

public boolean add(long ledgerId, long entryId, long stickyKeyHash) {
if (hashesToBeBlocked != null) {
hashesToBeBlocked.put(ledgerId, entryId, stickyKeyHash, 0);
}
return messagesToRedeliver.add(ledgerId, entryId);
}

public boolean remove(long ledgerId, long entryId) {
if (hashesToBeBlocked != null) {
hashesToBeBlocked.remove(ledgerId, entryId);
}
return messagesToRedeliver.remove(ledgerId, entryId);
}

public int removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) {
if (hashesToBeBlocked != null) {
for (LongPair longPair : hashesToBeBlocked.keys()) {
long ledgerId = longPair.first;
long entryId = longPair.second;
if (ComparisonChain.start().compare(ledgerId, markDeleteLedgerId).compare(entryId, markDeleteEntryId)
.result() <= 0) {
hashesToBeBlocked.remove(ledgerId, entryId);
}
}
}
return messagesToRedeliver.removeIf((ledgerId, entryId) -> {
return ComparisonChain.start().compare(ledgerId, markDeleteLedgerId).compare(entryId, markDeleteEntryId)
.result() <= 0;
});
}

public boolean isEmpty() {
return messagesToRedeliver.isEmpty();
}

public void clear() {
if (hashesToBeBlocked != null) {
hashesToBeBlocked.clear();
}
messagesToRedeliver.clear();
}

public String toString() {
return messagesToRedeliver.toString();
}

public boolean containsStickyKeyHashes(Set<Integer> stickyKeyHashes) {
if (hashesToBeBlocked != null) {
for (LongPair longPair : hashesToBeBlocked.values()) {
int stickyKeyHash = (int) longPair.first;
if (stickyKeyHashes.contains(stickyKeyHash)) {
return true;
}
}
}
return false;
}

public Set<PositionImpl> getMessagesToReplayNow(int maxMessagesToRead) {
if (hashesToBeBlocked != null) {
return messagesToRedeliver.items().stream()
.sorted((l1, l2) -> ComparisonChain.start().compare(l1.first, l2.first)
.compare(l1.second, l2.second).result())
.limit(maxMessagesToRead).map(longPair -> new PositionImpl(longPair.first, longPair.second))
.collect(Collectors.toCollection(TreeSet::new));
} else {
return messagesToRedeliver.items(maxMessagesToRead,
(ledgerId, entryId) -> new PositionImpl(ledgerId, entryId));
}
}
}

0 comments on commit 401f4ee

Please sign in to comment.