Skip to content

Commit

Permalink
add a test
Browse files Browse the repository at this point in the history
  • Loading branch information
poorbarcode committed Apr 1, 2024
1 parent e7c6318 commit c163106
Show file tree
Hide file tree
Showing 2 changed files with 263 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,28 +18,56 @@
*/
package org.apache.pulsar.broker.service;

import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.spy;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;
import com.google.common.collect.Sets;
import io.netty.util.concurrent.FastThreadLocalThread;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
import java.time.Duration;
import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiFunction;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.SneakyThrows;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.broker.service.persistent.PersistentReplicator;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerBuilder;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.impl.ProducerBuilderImpl;
import org.apache.pulsar.client.impl.ProducerImpl;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
import org.apache.pulsar.common.policies.data.TopicStats;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.awaitility.Awaitility;
import org.awaitility.reflect.WhiteboxImpl;
import org.mockito.Mockito;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

@Slf4j
@Test(groups = "broker")
public class OneWayReplicatorTest extends OneWayReplicatorTestBase {

Expand Down Expand Up @@ -153,4 +181,225 @@ public void testTopicCloseWhenInternalProducerCloseErrorOnce() throws Exception
admin2.topics().delete(topicName);
});
}

private void injectMockReplicatorProducerBuilder(
BiFunction<ProducerConfigurationData, ProducerImpl, ProducerImpl> producerDecorator)
throws Exception {
String cluster2 = pulsar2.getConfig().getClusterName();
BrokerService brokerService = pulsar1.getBrokerService();
// Wait for the internal client created.
final String topicNameTriggerInternalClientCreate =
BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_");
admin1.topics().createNonPartitionedTopic(topicNameTriggerInternalClientCreate);
waitReplicatorStarted(topicNameTriggerInternalClientCreate);
cleanupTopics(() -> {
admin1.topics().delete(topicNameTriggerInternalClientCreate);
admin2.topics().delete(topicNameTriggerInternalClientCreate);
});

// Inject spy client.
ConcurrentOpenHashMap<String, PulsarClient>
replicationClients = WhiteboxImpl.getInternalState(brokerService, "replicationClients");
PulsarClientImpl internalClient = (PulsarClientImpl) replicationClients.get(cluster2);
PulsarClient spyClient = spy(internalClient);
replicationClients.put(cluster2, spyClient);

// Inject producer decorator.
doAnswer(invocation -> {
Schema schema = (Schema) invocation.getArguments()[0];
ProducerBuilderImpl producerBuilder = (ProducerBuilderImpl) internalClient.newProducer(schema);
ProducerBuilder spyProducerBuilder = spy(producerBuilder);
doAnswer(ignore -> {
CompletableFuture<Producer> producerFuture = new CompletableFuture<>();
final ProducerImpl p = (ProducerImpl) producerBuilder.create();
new FastThreadLocalThread(() -> {
try {
ProducerImpl newProducer = producerDecorator.apply(producerBuilder.getConf(), p);
producerFuture.complete(newProducer);
} catch (Exception ex) {
producerFuture.completeExceptionally(ex);
}
}).start();
return producerFuture;
}).when(spyProducerBuilder).createAsync();
return spyProducerBuilder;
}).when(spyClient).newProducer(any(Schema.class));
}

private SpyCursor spyCursor(PersistentTopic persistentTopic, String cursorName) throws Exception {
ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger();
ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(cursorName);
ManagedCursorImpl spyCursor = spy(cursor);
// remove cursor.
ml.getCursors().removeCursor(cursorName);
ml.deactivateCursor(cursor);
// Add the spy one. addCursor(ManagedCursorImpl cursor)
Method m = ManagedLedgerImpl.class.getDeclaredMethod("addCursor", new Class[]{ManagedCursorImpl.class});
m.setAccessible(true);
m.invoke(ml, new Object[]{spyCursor});
return new SpyCursor(cursor, spyCursor);
}

@Data
@AllArgsConstructor
static class SpyCursor {
ManagedCursorImpl original;
ManagedCursorImpl spy;
}

private CursorCloseSignal makeCursorClosingDelay(SpyCursor spyCursor) throws Exception {
CountDownLatch startCloseSignal = new CountDownLatch(1);
CountDownLatch startCallbackSignal = new CountDownLatch(1);
doAnswer(invocation -> {
AsyncCallbacks.CloseCallback originalCallback = (AsyncCallbacks.CloseCallback) invocation.getArguments()[0];
Object ctx = invocation.getArguments()[1];
AsyncCallbacks.CloseCallback newCallback = new AsyncCallbacks.CloseCallback() {
@Override
public void closeComplete(Object ctx) {
new FastThreadLocalThread(new Runnable() {
@Override
@SneakyThrows
public void run() {
startCallbackSignal.await();
originalCallback.closeComplete(ctx);
}
}).start();
}

@Override
public void closeFailed(ManagedLedgerException exception, Object ctx) {
new FastThreadLocalThread(new Runnable() {
@Override
@SneakyThrows
public void run() {
startCallbackSignal.await();
originalCallback.closeFailed(exception, ctx);
}
}).start();
}
};
startCloseSignal.await();
spyCursor.original.asyncClose(newCallback, ctx);
return null;
}).when(spyCursor.spy).asyncClose(any(AsyncCallbacks.CloseCallback.class), any());
return new CursorCloseSignal(startCloseSignal, startCallbackSignal);
}

@AllArgsConstructor
static class CursorCloseSignal {
CountDownLatch startCloseSignal;
CountDownLatch startCallbackSignal;

void startClose() {
startCloseSignal.countDown();
}

void startCallback() {
startCallbackSignal.countDown();
}
}

/**
* See the description and execution flow: https://github.com/apache/pulsar/pull/21948.
* Steps:
* 1.Create topic, does not enable replication now.
* - The topic will be loaded in the memory.
* 2.Enable namespace level replication.
* - Broker creates a replicator, and the internal producer of replicator is starting.
* - We inject an error to make the internal producer fail to connect,after few seconds, it will retry to start.
* 3.Unload bundle.
* - Starting to close the topic.
* - The replicator will be closed, but it will not close the internal producer, because the producer has not
* been created successfully.
* - We inject a sleeping into the progress of closing the "repl.cursor" to make it stuck. So the topic is still
* in the process of being closed now.
* 4.Internal producer retry to connect.
* - At the next retry, it connected successful. Since the state of "repl.cursor" is not "Closed", this producer
* will not be closed now.
* 5.Topic closed.
* - Cancel the stuck of closing the "repl.cursor".
* - The topic is wholly closed.
* 6.Verify: the delayed created internal producer will be closed. In other words, there is no producer is connected
* to the remote cluster.
*/
@Test
public void testConcurrencyOfUnloadBundleAndRecreateProducer2() throws Exception {
final String namespaceName = defaultTenant + "/" + UUID.randomUUID().toString().replaceAll("-", "");
final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespaceName + "/tp_");
// 1.Create topic, does not enable replication now.
admin1.namespaces().createNamespace(namespaceName);
admin2.namespaces().createNamespace(namespaceName);
admin1.topics().createNonPartitionedTopic(topicName);
PersistentTopic persistentTopic =
(PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get();

// We inject an error to make the internal producer fail to connect.
// The delay time of next retry to create producer is below:
// 0.1s, 0.2, 0.4, 0.8, 1.6s, 3.2s, 6.4s...
// If the retry counter is larger than 6, the next creation will be slow enough to close Replicator.
final AtomicInteger createProducerCounter = new AtomicInteger();
final int failTimes = 6;
injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> {
if (topicName.equals(producerCnf.getTopicName())) {
// There is a switch to determine create producer successfully or not.
if (createProducerCounter.incrementAndGet() > failTimes) {
return originalProducer;
}
log.info("Retry create replicator.producer count: {}", createProducerCounter);
// Release producer and fail callback.
originalProducer.closeAsync();
throw new RuntimeException("mock error");
}
return originalProducer;
});

// 2.Enable namespace level replication.
admin1.namespaces().setNamespaceReplicationClusters(namespaceName, Sets.newHashSet(cluster1, cluster2));
AtomicReference<PersistentReplicator> replicator = new AtomicReference<PersistentReplicator>();
Awaitility.await().untilAsserted(() -> {
assertFalse(persistentTopic.getReplicators().isEmpty());
replicator.set(
(PersistentReplicator) persistentTopic.getReplicators().values().iterator().next());
// Since we inject a producer creation error, the replicator can not start successfully.
assertFalse(replicator.get().isConnected());
});

// We inject a sleeping into the progress of closing the "repl.cursor" to make it stuck, until the internal
// producer of the replicator started.
SpyCursor spyCursor =
spyCursor(persistentTopic, "pulsar.repl." + pulsar2.getConfig().getClusterName());
CursorCloseSignal cursorCloseSignal = makeCursorClosingDelay(spyCursor);

// 3.Unload bundle: call "topic.close(false)".
// Stuck start new producer, until the state of replicator change to Stopped.
// The next once of "createProducerSuccessAfterFailTimes" to create producer will be successfully.
Awaitility.await().pollInterval(Duration.ofMillis(100)).atMost(Duration.ofSeconds(60)).untilAsserted(() -> {
assertTrue(createProducerCounter.get() >= failTimes);
});
CompletableFuture<Void> topicCloseFuture = persistentTopic.close(true);
Awaitility.await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> {
String state = String.valueOf(replicator.get().getState());
assertTrue(state.equals("Stopped") || state.equals("Terminated"));
});

// 5.Delay close cursor, until "replicator.producer" create successfully.
// The next once retry time of create "replicator.producer" will be 3.2s.
Thread.sleep(4 * 1000);
log.info("Replicator.state: {}", replicator.get().getState());
cursorCloseSignal.startClose();
cursorCloseSignal.startCallback();
// Wait for topic close successfully.
topicCloseFuture.join();

// 6. Verify there is no orphan producer on the remote cluster.
Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(() -> {
PersistentTopic persistentTopic2 =
(PersistentTopic) pulsar2.getBrokerService().getTopic(topicName, false).join().get();
assertEquals(persistentTopic2.getProducers().size(), 0);
Assert.assertFalse(replicator.get().isConnected());
});

// cleanup.
cleanupNamespace(namespaceName);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.google.common.collect.Sets;
import java.net.URL;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.broker.PulsarService;
Expand Down Expand Up @@ -146,6 +147,19 @@ protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Excep
admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster1, cluster2));
}

protected void cleanupNamespace(String namespace) throws Exception {
admin1.namespaces().setNamespaceReplicationClusters(namespace, Collections.singleton(cluster1));
List<String> partitionedTopics = admin1.topics().getPartitionedTopicList(namespace);
for (String partitionedTopic: partitionedTopics) {
admin1.topics().deletePartitionedTopic(partitionedTopic);
}
List<String> nonPartitionedTopics = admin1.topics().getList(namespace);
for (String nonPartitionedTopic: nonPartitionedTopics) {
admin1.topics().delete(nonPartitionedTopic);
}
admin1.namespaces().deleteNamespace(namespace);
}

protected interface CleanupTopicAction {
void run() throws Exception;
}
Expand Down

0 comments on commit c163106

Please sign in to comment.