lhotari commented on code in PR #23352:
URL: https://github.com/apache/pulsar/pull/23352#discussion_r1797026999


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java:
##########
@@ -131,60 +131,74 @@ public synchronized CompletableFuture<Void> 
addConsumer(Consumer consumer) {
             consumer.disconnect();
             return CompletableFuture.completedFuture(null);
         }
-        return super.addConsumer(consumer).thenCompose(__ ->
-                selector.addConsumer(consumer).handle((result, ex) -> {
-                    if (ex != null) {
-                        synchronized 
(PersistentStickyKeyDispatcherMultipleConsumers.this) {
-                            consumerSet.removeAll(consumer);
-                            consumerList.remove(consumer);
-                        }
-                        throw FutureUtil.wrapToCompletionException(ex);
+        return super.addConsumer(consumer).thenCompose(__ -> 
selector.addConsumer(consumer))
+                .thenAccept(impactedConsumers -> {
+            // TODO: Add some way to prevent changes in between the time the 
consumer is added and the
+            // time the draining hashes are applied. It might be fine for 
ConsistentHashingStickyKeyConsumerSelector
+            // since it's not really asynchronous, although it returns a 
CompletableFuture
+            if (drainingHashesRequired) {
+                
consumer.setPendingAcksAddHandler(this::handleAddingPendingAck);
+                consumer.setPendingAcksRemoveHandler(new 
PendingAcksMap.PendingAcksRemoveHandler() {
+                    @Override
+                    public void handleRemoving(Consumer consumer, long 
ledgerId, long entryId, int stickyKeyHash,
+                                               boolean closing) {
+                        drainingHashesTracker.reduceRefCount(consumer, 
stickyKeyHash, closing);
                     }
-                    return result;
-                })
-        ).thenRun(() -> {
-            synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) 
{
-                if (recentlyJoinedConsumerTrackingRequired) {
-                    final Position lastSentPositionWhenJoining = 
updateIfNeededAndGetLastSentPosition();
-                    if (lastSentPositionWhenJoining != null) {
-                        
consumer.setLastSentPositionWhenJoining(lastSentPositionWhenJoining);
-                        // If this was the 1st consumer, or if all the 
messages are already acked, then we
-                        // don't need to do anything special
-                        if (recentlyJoinedConsumers != null
-                                && consumerList.size() > 1
-                                && 
cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) {
-                            recentlyJoinedConsumers.put(consumer, 
lastSentPositionWhenJoining);
-                        }
+
+                    @Override
+                    public void startBatch() {
+                        drainingHashesTracker.startBatch();
                     }
-                }
+
+                    @Override
+                    public void endBatch() {
+                        drainingHashesTracker.endBatch();
+                    }
+                });
+                registerDrainingHashes(consumer, impactedConsumers);
+            }
+        }).exceptionally(ex -> {
+            internalRemoveConsumer(consumer);
+            throw FutureUtil.wrapToCompletionException(ex);
+        });
+    }
+
+    private synchronized void registerDrainingHashes(Consumer skipConsumer,
+                                                     ImpactedConsumersResult 
impactedConsumers) {
+        impactedConsumers.processRemovedHashRanges((c, removedHashRanges) -> {
+            if (c != skipConsumer) {

Review Comment:
   > What happens if a hash is moved between existing consumers? Is it handled?
   
   Yes. All removed ranges are processed in both when a consumer is added and 
when it's removed. In some experimental testing I noticed that it actually 
could happen also when a consumer is added, due to how hash range conflicts are 
resolved. There's a need to make an improvement that when there are conflicts 
that the first assigned consumer is preserved instead of replacing it. I'll 
make a PR for that since hash range conflicts are fairly likely to happen with 
the reduced hash range of 65535 and 100 hash points when there's a large amount 
of consumers. I'll address that.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to