Skip to content

Commit

Permalink
[fix][broker] Fix the publish latency spike from the contention of Me…
Browse files Browse the repository at this point in the history
…ssageDeduplication (#20647)
  • Loading branch information
codelipenghui authored and Technoboy- committed Jun 29, 2023
1 parent 39cf8d7 commit fa68bf3
Show file tree
Hide file tree
Showing 2 changed files with 7 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,12 @@

import com.google.common.annotations.VisibleForTesting;
import io.netty.buffer.ByteBuf;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCursorCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback;
Expand Down Expand Up @@ -126,7 +126,7 @@ public MessageDupUnknownException() {
private final int maxNumberOfProducers;

// Map used to track the inactive producer along with the timestamp of their last activity
private final Map<String, Long> inactiveProducers = new HashMap<>();
private final Map<String, Long> inactiveProducers = new ConcurrentHashMap<>();

private final String replicatorPrefix;

Expand Down Expand Up @@ -436,15 +436,15 @@ private boolean isDeduplicationEnabled() {
/**
* Topic will call this method whenever a producer connects.
*/
public synchronized void producerAdded(String producerName) {
public void producerAdded(String producerName) {
// Producer is no-longer inactive
inactiveProducers.remove(producerName);
}

/**
* Topic will call this method whenever a producer disconnects.
*/
public synchronized void producerRemoved(String producerName) {
public void producerRemoved(String producerName) {
// Producer is no-longer active
inactiveProducers.put(producerName, System.currentTimeMillis());
}
Expand All @@ -456,7 +456,7 @@ public synchronized void purgeInactiveProducers() {
long minimumActiveTimestamp = System.currentTimeMillis() - TimeUnit.MINUTES
.toMillis(pulsar.getConfiguration().getBrokerDeduplicationProducerInactivityTimeoutMinutes());

Iterator<java.util.Map.Entry<String, Long>> mapIterator = inactiveProducers.entrySet().iterator();
Iterator<Map.Entry<String, Long>> mapIterator = inactiveProducers.entrySet().iterator();
boolean hasInactive = false;
while (mapIterator.hasNext()) {
java.util.Map.Entry<String, Long> entry = mapIterator.next();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import io.netty.channel.EventLoopGroup;
import java.lang.reflect.Field;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedLedger;
Expand Down Expand Up @@ -174,7 +175,7 @@ public void testInactiveProducerRemove() throws Exception {

Field field = MessageDeduplication.class.getDeclaredField("inactiveProducers");
field.setAccessible(true);
Map<String, Long> inactiveProducers = (Map<String, Long>) field.get(messageDeduplication);
Map<String, Long> inactiveProducers = (ConcurrentHashMap<String, Long>) field.get(messageDeduplication);

String producerName1 = "test1";
when(publishContext.getHighestSequenceId()).thenReturn(2L);
Expand Down

0 comments on commit fa68bf3

Please sign in to comment.