Skip to content
New issue

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

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

Already on GitHub? # to your account

[fix] [broker] fix write all compacted out entry into compacted topic #21917

Merged
merged 10 commits into from
Jan 21, 2024
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 @@ -134,7 +134,11 @@ public static Optional<RawMessage> rebatchMessage(RawMessage msg,
msg.getMessageIdData().getEntryId(),
msg.getMessageIdData().getPartition(),
i);
if (!singleMessageMetadata.hasPartitionKey()) {
if (singleMessageMetadata.isCompactedOut()) {
// we may read compacted out message from the compacted topic
Commands.serializeSingleMessageInBatchWithPayload(emptyMetadata,
Unpooled.EMPTY_BUFFER, batchBuffer);
} else if (!singleMessageMetadata.hasPartitionKey()) {
if (retainNullKey) {
messagesRetained++;
Commands.serializeSingleMessageInBatchWithPayload(singleMessageMetadata,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;

import lombok.Cleanup;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.LedgerEntry;
Expand All @@ -46,10 +48,15 @@
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.admin.LongRunningProcessStatus;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.MessageRoutingMode;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.RawMessage;
import org.apache.pulsar.client.api.Reader;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.impl.ConnectionPool;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.client.impl.RawMessageImpl;
Expand Down Expand Up @@ -177,6 +184,55 @@ public void testCompaction() throws Exception {
compactAndVerify(topic, expected, true);
}

@Test
public void testAllCompactedOut() throws Exception {
String topicName = "persistent://my-property/use/my-ns/testAllCompactedOut";
// set retain null key to true
boolean oldRetainNullKey = pulsar.getConfig().isTopicCompactionRetainNullKey();
pulsar.getConfig().setTopicCompactionRetainNullKey(true);
this.restartBroker();

@Cleanup
Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
.enableBatching(true).topic(topicName).batchingMaxMessages(3).create();

producer.newMessage().key("K1").value("V1").sendAsync();
producer.newMessage().key("K2").value("V2").sendAsync();
producer.newMessage().key("K2").value(null).sendAsync();
producer.flush();

admin.topics().triggerCompaction(topicName);

Awaitility.await().untilAsserted(() -> {
Assert.assertEquals(admin.topics().compactionStatus(topicName).status,
LongRunningProcessStatus.Status.SUCCESS);
});

producer.newMessage().key("K1").value(null).sendAsync();
producer.flush();

admin.topics().triggerCompaction(topicName);

Awaitility.await().untilAsserted(() -> {
Assert.assertEquals(admin.topics().compactionStatus(topicName).status,
LongRunningProcessStatus.Status.SUCCESS);
});

@Cleanup
Reader<String> reader = pulsarClient.newReader(Schema.STRING)
.subscriptionName("reader-test")
.topic(topicName)
.readCompacted(true)
.startMessageId(MessageId.earliest)
.create();
while (reader.hasMessageAvailable()) {
Message<String> message = reader.readNext(3, TimeUnit.SECONDS);
Assert.assertNotNull(message);
}
// set retain null key back to avoid affecting other tests
pulsar.getConfig().setTopicCompactionRetainNullKey(oldRetainNullKey);
}

@Test
public void testCompactAddCompact() throws Exception {
String topic = "persistent://my-property/use/my-ns/my-topic1";
Expand Down
Loading