Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix][broker] fix replicated subscriptions for transactional messages #22452

Merged
Merged
Show file tree
Hide file tree
Changes from 9 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -3942,9 +3942,13 @@ public void publishTxnMessage(TxnID txnID, ByteBuf headersAndPayload, PublishCon
@Override
public CompletableFuture<Void> endTxn(TxnID txnID, int txnAction, long lowWaterMark) {
if (TxnAction.COMMIT_VALUE == txnAction) {
return transactionBuffer.commitTxn(txnID, lowWaterMark);
return transactionBuffer.commitTxn(txnID, lowWaterMark).thenRun(() -> {
lastDataMessagePublishedTimestamp = Clock.systemUTC().millis();
thetumbled marked this conversation as resolved.
Show resolved Hide resolved
});
} else if (TxnAction.ABORT_VALUE == txnAction) {
return transactionBuffer.abortTxn(txnID, lowWaterMark);
return transactionBuffer.abortTxn(txnID, lowWaterMark).thenRun(() -> {
lastDataMessagePublishedTimestamp = Clock.systemUTC().millis();
thetumbled marked this conversation as resolved.
Show resolved Hide resolved
thetumbled marked this conversation as resolved.
Show resolved Hide resolved
});
} else {
return FutureUtil.failedFuture(new NotAllowedException("Unsupported txnAction " + txnAction));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -298,6 +298,8 @@ public void testPublishMessage() throws Exception {
}).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), any(AddEntryCallback.class), any());

PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService);
long lastDataMessagePublishedTimestamp = topic.getLastDataMessagePublishedTimestamp();

/*
* MessageMetadata.Builder messageMetadata = MessageMetadata.newBuilder();
* messageMetadata.setPublishTime(System.currentTimeMillis()); messageMetadata.setProducerName("producer-name");
Expand All @@ -322,10 +324,10 @@ public void setMetadataFromEntryData(ByteBuf entryData) {
assertEquals(entryData.array(), payload.array());
}
};

topic.publishMessage(payload, publishContext);

assertTrue(latch.await(1, TimeUnit.SECONDS));
assertTrue(topic.getLastDataMessagePublishedTimestamp() > lastDataMessagePublishedTimestamp);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,174 @@
package org.apache.pulsar.broker.service;
thetumbled marked this conversation as resolved.
Show resolved Hide resolved

import com.google.common.collect.Sets;
import lombok.Cleanup;
import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageRoutingMode;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.transaction.Transaction;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.SystemTopicNames;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.LinkedHashSet;
import java.util.Set;
import java.util.concurrent.TimeUnit;

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;

public class TransactionalReplicateSubscriptionTest extends ReplicatorTestBase {
@Override
@BeforeClass(timeOut = 300000)
public void setup() throws Exception {
super.setup();
admin1.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString());
createTransactionCoordinatorAssign(16, pulsar1);
}

@Override
@AfterClass(alwaysRun = true, timeOut = 300000)
public void cleanup() throws Exception {
super.cleanup();
}

@Override
public void setConfig1DefaultValue(){
super.setConfig1DefaultValue();
config1.setTransactionCoordinatorEnabled(true);
}

protected void createTransactionCoordinatorAssign(int numPartitionsOfTC, PulsarService pulsarService) throws MetadataStoreException {
pulsarService.getPulsarResources()
.getNamespaceResources()
.getPartitionedTopicResources()
.createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN,
new PartitionedTopicMetadata(numPartitionsOfTC));
}

void createReplicatedSubscription(PulsarClient pulsarClient, String topicName, String subscriptionName,
boolean replicateSubscriptionState)
throws PulsarClientException {
pulsarClient.newConsumer().topic(topicName)
.subscriptionName(subscriptionName)
.replicateSubscriptionState(replicateSubscriptionState)
.subscribe()
.close();
}

/**
* Tests replicated subscriptions across two regions with transactional production
*/
@Test
public void testReplicatedSubscriptionAcrossTwoRegionsWithTransactionalProduction() throws Exception {
String namespace = BrokerTestUtil.newUniqueName("pulsar/replicatedsubscription");
String topicName = "persistent://" + namespace + "/mytopic2";
String subscriptionName = "cluster-subscription2";
// Subscription replication produces duplicates, https://github.com/apache/pulsar/issues/10054
// TODO: duplications shouldn't be allowed, change to "false" when fixing the issue
boolean allowDuplicates = true;
// this setting can be used to manually run the test with subscription replication disabled
// it shows that subscription replication has no impact in behavior for this test case
boolean replicateSubscriptionState = true;

admin1.namespaces().createNamespace(namespace);
admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2"));

@Cleanup
PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString())
.statsInterval(0, TimeUnit.SECONDS)
.enableTransaction(true)
.build();

// create subscription in r1
createReplicatedSubscription(client1, topicName, subscriptionName, replicateSubscriptionState);

@Cleanup
PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString())
.statsInterval(0, TimeUnit.SECONDS)
.build();

// create subscription in r2
createReplicatedSubscription(client2, topicName, subscriptionName, replicateSubscriptionState);

Set<String> sentMessages = new LinkedHashSet<>();

// send messages in r1
{
@Cleanup
Producer<byte[]> producer = client1.newProducer().topic(topicName)
.enableBatching(false)
.messageRoutingMode(MessageRoutingMode.SinglePartition)
.create();
Transaction txn = client1.newTransaction()
.withTransactionTimeout(5, TimeUnit.SECONDS)
.build().get();
int numMessages = 6;
for (int i = 0; i < numMessages; i++) {
String body = "message" + i;
producer.newMessage(txn).value(body.getBytes(StandardCharsets.UTF_8)).send();
sentMessages.add(body);
}
txn.commit().get();
producer.close();
}

Set<String> receivedMessages = new LinkedHashSet<>();

// consume 3 messages in r1
try (Consumer<byte[]> consumer1 = client1.newConsumer()
.topic(topicName)
.subscriptionName(subscriptionName)
.replicateSubscriptionState(replicateSubscriptionState)
.subscribe()) {
readMessages(consumer1, receivedMessages, 3, allowDuplicates);
}

// wait for subscription to be replicated
Thread.sleep(2 * config1.getReplicatedSubscriptionsSnapshotFrequencyMillis());

// consume remaining messages in r2
try (Consumer<byte[]> consumer2 = client2.newConsumer()
.topic(topicName)
.subscriptionName(subscriptionName)
.replicateSubscriptionState(replicateSubscriptionState)
.subscribe()) {
readMessages(consumer2, receivedMessages, -1, allowDuplicates);
}

// assert that all messages have been received
assertEquals(new ArrayList<>(sentMessages), new ArrayList<>(receivedMessages), "Sent and received " +
"messages don't match.");
}

int readMessages(Consumer<byte[]> consumer, Set<String> messages, int maxMessages, boolean allowDuplicates)
throws PulsarClientException {
int count = 0;
while (count < maxMessages || maxMessages == -1) {
Message<byte[]> message = consumer.receive(2, TimeUnit.SECONDS);
if (message != null) {
count++;
String body = new String(message.getValue(), StandardCharsets.UTF_8);
if (!allowDuplicates) {
assertFalse(messages.contains(body), "Duplicate message '" + body + "' detected.");
}
messages.add(body);
consumer.acknowledge(message);
} else {
break;
}
}
return count;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,14 @@
package org.apache.pulsar.broker.transaction;

import static java.nio.charset.StandardCharsets.UTF_8;
import static org.testng.Assert.assertTrue;

import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
Expand All @@ -38,7 +41,9 @@
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.commons.lang3.tuple.MutablePair;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
Expand Down Expand Up @@ -180,6 +185,37 @@ private void produceTest(boolean endAction) throws Exception {
log.info("produce and {} test finished.", endAction ? "commit" : "abort");
}

@Test
public void testUpdateLastDataMessagePublishedTimestampForTransactionalPublish() throws Exception {
final String topic = NAMESPACE1 + "/testUpdateLastDataMessagePublishedTimestampForTransactionalPublish";
PulsarClient pulsarClient = this.pulsarClient;
Transaction txn = pulsarClient.newTransaction()
.withTransactionTimeout(5, TimeUnit.SECONDS)
.build().get();
@Cleanup
Producer<byte[]> producer = pulsarClient
.newProducer()
.topic(topic)
.sendTimeout(0, TimeUnit.SECONDS)
.create();
PersistentTopic persistentTopic = getTopic(topic);
long lastDataMessagePublishedTimestamp = persistentTopic.getLastDataMessagePublishedTimestamp();

// transactional publish will not update lastDataMessagePublishedTimestamp
producer.newMessage(txn).value("hello world".getBytes()).send();
assertTrue(persistentTopic.getLastDataMessagePublishedTimestamp() == lastDataMessagePublishedTimestamp);

// commit transaction will update lastDataMessagePublishedTimestamp
txn.commit().get();
assertTrue(persistentTopic.getLastDataMessagePublishedTimestamp() > lastDataMessagePublishedTimestamp);
}

private PersistentTopic getTopic(String topic) throws ExecutionException, InterruptedException {
Optional<Topic> optionalTopic = getPulsarServiceList().get(0).getBrokerService()
.getTopic(topic, true).get();
return (PersistentTopic) optionalTopic.get();
}

private void checkMessageId(List<CompletableFuture<MessageId>> futureList, boolean isFinished) {
futureList.forEach(messageIdFuture -> {
try {
Expand Down
Loading