Skip to content

Commit

Permalink
[fix][broker] Fix pulsar.replicated.subscription checks (#23782)
Browse files Browse the repository at this point in the history
Signed-off-by: Zixuan Liu <[email protected]>
  • Loading branch information
nodece authored Dec 31, 2024
1 parent fdfe00a commit 149b9f7
Show file tree
Hide file tree
Showing 3 changed files with 50 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ public class PersistentSubscription extends AbstractSubscription {
// for connected subscriptions, message expiry will be checked if the backlog is greater than this threshold
private static final int MINIMUM_BACKLOG_FOR_EXPIRY_CHECK = 1000;

private static final String REPLICATED_SUBSCRIPTION_PROPERTY = "pulsar.replicated.subscription";
protected static final String REPLICATED_SUBSCRIPTION_PROPERTY = "pulsar.replicated.subscription";

// Map of properties that is used to mark this subscription as "replicated".
// Since this is the only field at this point, we can just keep a static
Expand All @@ -140,8 +140,12 @@ static Map<String, Long> getBaseCursorProperties(Boolean isReplicated) {
NON_REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES;
}

static boolean isCursorFromReplicatedSubscription(ManagedCursor cursor) {
return cursor.getProperties().containsKey(REPLICATED_SUBSCRIPTION_PROPERTY);
static Optional<Boolean> getReplicatedSubscriptionConfiguration(ManagedCursor cursor) {
Long v = cursor.getProperties().get(REPLICATED_SUBSCRIPTION_PROPERTY);
if (v == null || (v < 0L || v > 1L)) {
return Optional.empty();
}
return Optional.of(v == 1L);
}

public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -512,9 +512,12 @@ private void createPersistentSubscriptions() {
// ignore it for now and let the message dedup logic to take care of it
} else {
final String subscriptionName = Codec.decode(cursor.getName());
subscriptions.put(subscriptionName, createPersistentSubscription(subscriptionName, cursor,
PersistentSubscription.isCursorFromReplicatedSubscription(cursor) ? true : null,
cursor.getCursorProperties()));
Optional<Boolean> replicatedSubscriptionConfiguration =
PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor);
Boolean replicated = replicatedSubscriptionConfiguration.orElse(null);
subscriptions.put(subscriptionName,
createPersistentSubscription(subscriptionName, cursor, replicated,
cursor.getCursorProperties()));
// subscription-cursor gets activated by default: deactivate as there is no active subscription
// right now
subscriptions.get(subscriptionName).deactivateCursor();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.service.persistent;

import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doCallRealMethod;
Expand All @@ -30,11 +31,13 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
import org.apache.bookkeeper.mledger.Position;
Expand Down Expand Up @@ -226,6 +229,40 @@ public void testAcknowledgeUpdateCursorLastActive() throws Exception {
assertTrue(persistentSubscription.cursor.getLastActive() > beforeAcknowledgeTimestamp);
}

@Test
public void testGetReplicatedSubscriptionConfiguration() {
Map<String, Long> properties = PersistentSubscription.getBaseCursorProperties(true);
assertThat(properties).containsEntry(PersistentSubscription.REPLICATED_SUBSCRIPTION_PROPERTY, 1L);
ManagedCursor cursor = mock(ManagedCursor.class);
doReturn(properties).when(cursor).getProperties();
Optional<Boolean> replicatedSubscriptionConfiguration =
PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor);
assertThat(replicatedSubscriptionConfiguration).isNotEmpty().get().isEqualTo(Boolean.TRUE);

properties = Map.of(PersistentSubscription.REPLICATED_SUBSCRIPTION_PROPERTY, 10L);
doReturn(properties).when(cursor).getProperties();
replicatedSubscriptionConfiguration =
PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor);
assertThat(replicatedSubscriptionConfiguration).isEmpty();
properties = Map.of(PersistentSubscription.REPLICATED_SUBSCRIPTION_PROPERTY, -1L);
doReturn(properties).when(cursor).getProperties();
replicatedSubscriptionConfiguration =
PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor);
assertThat(replicatedSubscriptionConfiguration).isEmpty();

properties = PersistentSubscription.getBaseCursorProperties(false);
doReturn(properties).when(cursor).getProperties();
replicatedSubscriptionConfiguration =
PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor);
assertThat(replicatedSubscriptionConfiguration).isEmpty();

properties = PersistentSubscription.getBaseCursorProperties(null);
doReturn(properties).when(cursor).getProperties();
replicatedSubscriptionConfiguration =
PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor);
assertThat(replicatedSubscriptionConfiguration).isEmpty();
}

public static class CustomTransactionPendingAckStoreProvider implements TransactionPendingAckStoreProvider {
@Override
public CompletableFuture<PendingAckStore> newPendingAckStore(PersistentSubscription subscription) {
Expand Down

0 comments on commit 149b9f7

Please sign in to comment.