Skip to content
Open
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 @@ -134,8 +134,6 @@ protected boolean replicateEntries(List<Entry> entries, InFlightTask inFlightTas
msg.getMessageBuilder().addProperty().setKey(MSG_PROP_REPL_SOURCE_POSITION)
.setValue(String.format("%s:%s", entry.getLedgerId(), entry.getEntryId()));

headersAndPayload.retain();

// Increment pending messages for messages produced locally
producer.sendAsync(msg, ProducerSendCallback.create(this, entry, msg, inFlightTask));
atLeastOneMessageSentForReplication = true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,20 @@
package org.apache.pulsar.broker.service.persistent;

import static org.apache.pulsar.broker.service.persistent.BrokerServicePersistInternalMethodInvoker.ensureNoBacklogByInflightTask;
import static org.apache.pulsar.common.protocol.Commands.serializeMetadataAndPayload;
import static org.testng.Assert.assertTrue;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import lombok.Cleanup;
import lombok.CustomLog;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.impl.EntryImpl;
import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.service.BrokerTestBase;
Expand All @@ -38,9 +44,11 @@
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.SubscriptionInitialPosition;
import org.apache.pulsar.client.api.schema.GenericRecord;
import org.apache.pulsar.common.api.proto.MessageMetadata;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ReplicatorStats;
import org.apache.pulsar.common.policies.data.TenantInfoImpl;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.schema.Schemas;
import org.awaitility.Awaitility;
import org.testng.Assert;
Expand Down Expand Up @@ -146,6 +154,39 @@ public void testShadowReplication() throws Exception {
Assert.assertEquals(shadowMessage.getMessageId(), sourceMessage.getMessageId());
}

@Test
public void testShadowReplicatorReleasesSourceEntryBuffer() throws Exception {
String sourceTopicName = BrokerTestUtil.newUniqueName("persistent://prop1/ns-source/source-topic");
String shadowTopicName = BrokerTestUtil.newUniqueName("persistent://prop1/ns-shadow/shadow-topic");

admin.topics().createNonPartitionedTopic(sourceTopicName);
admin.topics().createShadowTopic(shadowTopicName, sourceTopicName);
admin.topics().setShadowTopics(sourceTopicName, Lists.newArrayList(shadowTopicName));

PersistentTopic sourceTopic =
(PersistentTopic) pulsar.getBrokerService().getTopicIfExists(sourceTopicName).get().get();
Awaitility.await().untilAsserted(() -> Assert.assertEquals(sourceTopic.getShadowReplicators().size(), 1));
ShadowReplicator replicator = (ShadowReplicator) sourceTopic.getShadowReplicators().get(shadowTopicName);
Awaitility.await().untilAsserted(() ->
Assert.assertEquals(String.valueOf(replicator.getState()), "Started"));

Entry entry = createEntry(1, 0, "ref-count-check", 1);
ByteBuf entryBuffer = entry.getDataBuffer();
Assert.assertEquals(entryBuffer.refCnt(), 1);

List<Entry> entries = Lists.newArrayList(entry);
PersistentReplicator.InFlightTask inFlightTask =
new PersistentReplicator.InFlightTask(
entry.getPosition(), entries.size(), replicator.getReplicatorId());
inFlightTask.setEntries(entries);
Assert.assertTrue(replicator.replicateEntries(entries, inFlightTask));

Awaitility.await().untilAsserted(() -> {
Assert.assertTrue(inFlightTask.isDone());
Assert.assertEquals(entryBuffer.refCnt(), 0);
});
}

private static PersistentReplicator getAnyShadowReplicator(TopicName topicName, PulsarService pulsar) {
PersistentTopic persistentTopic =
(PersistentTopic) pulsar.getBrokerService().getTopic(topicName.toString(), false).join().get();
Expand Down Expand Up @@ -196,4 +237,22 @@ public void testCounterOfPendingMessagesCorrect() throws Exception {
ensureNoBacklogByInflightTask(replicator);
}

}
private Entry createEntry(long ledgerId, long entryId, String message, long sequenceId) {
ByteBuf headersAndPayload = createMessage(message, sequenceId);
Entry entry = EntryImpl.create(ledgerId, entryId, headersAndPayload);
headersAndPayload.release();
return entry;
}

private ByteBuf createMessage(String message, long sequenceId) {
MessageMetadata messageMetadata = new MessageMetadata()
.setSequenceId(sequenceId)
.setProducerName("testProducer")
.setPublishTime(System.currentTimeMillis());
ByteBuf payload = Unpooled.copiedBuffer(message.getBytes(StandardCharsets.UTF_8));
ByteBuf headersAndPayload = serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, payload);
payload.release();
return headersAndPayload;
}

}