Skip to content
Merged
Show file tree
Hide file tree
Changes from all 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 @@ -279,6 +279,9 @@ private AvailablePermits getRateLimiterAvailablePermits(int availablePermits) {
}

protected void readMoreEntries() {
if (state.equals(Terminated) || state.equals(Terminating)) {
return;
}
// Acquire permits and check state of producer.
InFlightTask newInFlightTask = acquirePermitsIfNotFetchingSchema();
if (newInFlightTask == null) {
Expand Down Expand Up @@ -963,12 +966,19 @@ public void beforeTerminate() {
protected boolean hasPendingRead() {
synchronized (inFlightTasks) {
for (InFlightTask task : inFlightTasks) {
if (task.readPos != null && task.entries == null) {
// The purpose of calling "getReadPos" instead of calling "readPos" is to make the test
// "testReplicationTaskStoppedAfterTopicClosed" can counter the calling times of "readMoreEntries".
if (task.getReadPos() != null && task.entries == null) {
// Skip the current reading if there is a pending cursor reading.
return true;
}
}
}
return false;
}

@VisibleForTesting
String getReplicatorId() {
return replicatorId;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,17 @@
*/
package org.apache.pulsar.broker.service.persistent;

import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.Position;
Expand All @@ -34,6 +39,8 @@
import org.apache.pulsar.broker.service.OneWayReplicatorTestBase;
import org.apache.pulsar.broker.service.persistent.PersistentReplicator.InFlightTask;
import org.apache.pulsar.client.api.MessageId;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
Expand Down Expand Up @@ -66,6 +73,40 @@ private void createTopics() throws Exception {
admin1.topics().createSubscription(topicName, subscriptionName, MessageId.earliest);
}

@Test
public void testReplicationTaskStoppedAfterTopicClosed() throws Exception {
// Close a topic, which has enabled replication.
final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_");
admin1.topics().createNonPartitionedTopic(topicName);
waitReplicatorStarted(topicName, pulsar2);
PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false)
.join().get();
PersistentReplicator replicator = (PersistentReplicator) topic.getReplicators().get(cluster2);
admin1.topics().unload(topicName);

// Inject a task into the "inFlightTasks" to calculate how many times the method "replicator.readMoreEntries"
// has been called.
AtomicInteger counter = new AtomicInteger();
InFlightTask injectedTask = new InFlightTask(PositionFactory.create(1, 1), 1, replicator.getReplicatorId());
injectedTask.setEntries(Collections.emptyList());
InFlightTask spyTask = spy(injectedTask);
replicator.inFlightTasks.add(spyTask);
doAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
counter.incrementAndGet();
return invocationOnMock.callRealMethod();
}
}).when(spyTask).getReadPos();

// Verify: there is no scheduled task to retry to read entries to replicate.
// Call "readMoreEntries" to make the issue happen.
replicator.readMoreEntries();
Thread.sleep(PersistentTopic.MESSAGE_RATE_BACKOFF_MS * 10);
assertEquals(replicator.getState(), AbstractReplicator.State.Terminated);
assertTrue(counter.get() <= 1);
}

@Test
public void testCreateOrRecycleInFlightTaskIntoQueue() throws Exception {
log.info("Starting testCreateOrRecycleInFlightTaskIntoQueue");
Expand Down
Loading