You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2018/03/28 00:05:32 UTC

[GitHub] merlimat closed pull request #1427: Add batching support to compaction

merlimat closed pull request #1427: Add batching support to compaction
URL: https://github.com/apache/incubator-pulsar/pull/1427
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
index 3e2d25969..f12870b7b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
@@ -21,8 +21,11 @@
 import com.google.common.collect.ImmutableMap;
 import io.netty.buffer.ByteBuf;
 
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -43,6 +46,7 @@
 import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.RawReader;
 import org.apache.pulsar.client.api.RawMessage;
+import org.apache.pulsar.client.impl.RawBatchConverter;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -83,7 +87,8 @@ public TwoPhaseCompactor(ServiceConfiguration conf,
                     if (exception != null) {
                         loopPromise.completeExceptionally(exception);
                     } else {
-                        log.info("Commencing phase one of compaction for {}, reading to {}", reader, lastMessageId);
+                        log.info("Commencing phase one of compaction for {}, reading to {}",
+                                 reader.getTopic(), lastMessageId);
                         phaseOneLoop(reader, Optional.empty(), lastMessageId, latestForKey, loopPromise);
                     }
                 });
@@ -108,8 +113,18 @@ private void phaseOneLoop(RawReader reader,
                             return;
                         }
                         MessageId id = m.getMessageId();
-                        String key = extractKey(m);
-                        latestForKey.put(key, id);
+                        if (RawBatchConverter.isBatch(m)) {
+                            try {
+                                RawBatchConverter.extractIdsAndKeys(m)
+                                    .forEach(e -> latestForKey.put(e.getRight(), e.getLeft()));
+                            } catch (IOException ioe) {
+                                log.info("Error decoding batch for message {}. Whole batch will be included in output",
+                                         id, ioe);
+                            }
+                        } else {
+                            String key = extractKey(m);
+                            latestForKey.put(key, id);
+                        }
 
                         if (id.compareTo(lastMessageId) == 0) {
                             loopPromise.complete(new PhaseOneResult(firstMessageId.orElse(id),
@@ -140,7 +155,7 @@ private void scheduleTimeout(CompletableFuture<RawMessage> future) {
 
         return createLedger(bk).thenCompose((ledger) -> {
                 log.info("Commencing phase two of compaction for {}, from {} to {}, compacting {} keys to ledger {}",
-                         reader, from, to, latestForKey.size(), ledger.getId());
+                         reader.getTopic(), from, to, latestForKey.size(), ledger.getId());
                 return phaseTwoSeekThenLoop(reader, from, to, latestForKey, bk, ledger);
             });
     }
@@ -179,41 +194,55 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map<String, MessageId>
                               LedgerHandle lh, Semaphore outstanding, CompletableFuture<Void> promise) {
         reader.readNextAsync().whenCompleteAsync(
                 (m, exception) -> {
-                    try {
-                        if (exception != null) {
-                            promise.completeExceptionally(exception);
-                            return;
-                        } else if (promise.isDone()) {
-                            return;
+                    if (exception != null) {
+                        promise.completeExceptionally(exception);
+                        return;
+                    } else if (promise.isDone()) {
+                        return;
+                    }
+                    MessageId id = m.getMessageId();
+                    Optional<RawMessage> messageToAdd = Optional.empty();
+                    if (RawBatchConverter.isBatch(m)) {
+                        try {
+                            messageToAdd = RawBatchConverter.rebatchMessage(
+                                    m, (key, subid) -> latestForKey.get(key).equals(subid));
+                        } catch (IOException ioe) {
+                            log.info("Error decoding batch for message {}. Whole batch will be included in output",
+                                     id, ioe);
+                            messageToAdd = Optional.of(m);
                         }
-                        MessageId id = m.getMessageId();
+                    } else {
                         String key = extractKey(m);
-
                         if (latestForKey.get(key).equals(id)) {
-
-                            outstanding.acquire();
-                            CompletableFuture<Void> addFuture = addToCompactedLedger(lh, m)
-                                .whenComplete((res, exception2) -> {
-                                        outstanding.release();
-                                        if (exception2 != null) {
-                                            promise.completeExceptionally(exception2);
-                                        }
-                                    });
-                            if (to.equals(id)) {
-                                addFuture.whenComplete((res, exception2) -> {
-                                        if (exception2 == null) {
-                                            promise.complete(null);
-                                        }
-                                    });
-                            }
+                            messageToAdd = Optional.of(m);
+                        } else {
+                            m.close();
                         }
-                        phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise);
-                    } catch (InterruptedException ie) {
-                        Thread.currentThread().interrupt();
-                        promise.completeExceptionally(ie);
-                    } finally {
-                        m.close();
                     }
+
+                    messageToAdd.ifPresent((toAdd) -> {
+                            try {
+                                outstanding.acquire();
+                                CompletableFuture<Void> addFuture = addToCompactedLedger(lh, toAdd)
+                                    .whenComplete((res, exception2) -> {
+                                            outstanding.release();
+                                            if (exception2 != null) {
+                                                promise.completeExceptionally(exception2);
+                                            }
+                                        });
+                                if (to.equals(id)) {
+                                    addFuture.whenComplete((res, exception2) -> {
+                                            if (exception2 == null) {
+                                                promise.complete(null);
+                                            }
+                                        });
+                                }
+                            } catch (InterruptedException ie) {
+                                Thread.currentThread().interrupt();
+                                promise.completeExceptionally(ie);
+                            }
+                        });
+                    phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise);
                 }, scheduler);
     }
 
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java
index 7a22bb1ff..b2c2bbeeb 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java
@@ -36,8 +36,10 @@
 import org.apache.pulsar.client.api.MessageId;
 import org.apache.pulsar.client.api.Producer;
 import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.impl.BatchMessageIdImpl;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.PropertyAdmin;
+
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
@@ -342,4 +344,102 @@ public void testFirstMessageRetained() throws Exception {
             Assert.assertEquals(message2.getMessageId(), messages.get(2).getMessageId());
         }
     }
+
+    @Test
+    public void testBatchMessageIdsDontChange() throws Exception {
+        String topic = "persistent://my-property/use/my-ns/my-topic1";
+
+        // subscribe before sending anything, so that we get all messages
+        pulsarClient.newConsumer().topic(topic).subscriptionName("sub1")
+            .readCompacted(true).subscribe().close();
+
+        try (Producer producer = pulsarClient.newProducer().topic(topic).maxPendingMessages(3)
+                .enableBatching(true).batchingMaxMessages(3)
+                .batchingMaxPublishDelay(1, TimeUnit.HOURS).create()) {
+            producer.sendAsync(MessageBuilder.create()
+                               .setKey("key1")
+                               .setContent("my-message-1".getBytes()).build());
+            producer.sendAsync(MessageBuilder.create()
+                               .setKey("key2")
+                               .setContent("my-message-2".getBytes()).build());
+            producer.sendAsync(MessageBuilder.create()
+                               .setKey("key2")
+                               .setContent("my-message-3".getBytes()).build()).get();
+        }
+
+        // Read messages before compaction to get ids
+        List<Message> messages = new ArrayList<>();
+        try (Consumer consumer = pulsarClient.newConsumer().topic(topic)
+             .subscriptionName("sub1").readCompacted(true).subscribe()) {
+            messages.add(consumer.receive());
+            messages.add(consumer.receive());
+            messages.add(consumer.receive());
+        }
+
+        // Ensure all messages are in same batch
+        Assert.assertEquals(((BatchMessageIdImpl)messages.get(0).getMessageId()).getLedgerId(),
+                            ((BatchMessageIdImpl)messages.get(1).getMessageId()).getLedgerId());
+        Assert.assertEquals(((BatchMessageIdImpl)messages.get(0).getMessageId()).getLedgerId(),
+                            ((BatchMessageIdImpl)messages.get(2).getMessageId()).getLedgerId());
+        Assert.assertEquals(((BatchMessageIdImpl)messages.get(0).getMessageId()).getEntryId(),
+                            ((BatchMessageIdImpl)messages.get(1).getMessageId()).getEntryId());
+        Assert.assertEquals(((BatchMessageIdImpl)messages.get(0).getMessageId()).getEntryId(),
+                            ((BatchMessageIdImpl)messages.get(2).getMessageId()).getEntryId());
+
+        // compact the topic
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler);
+        compactor.compact(topic).get();
+
+        // Check that messages after compaction have same ids
+        try (Consumer consumer = pulsarClient.newConsumer().topic(topic)
+                .subscriptionName("sub1").readCompacted(true).subscribe()){
+            Message message1 = consumer.receive();
+            Assert.assertEquals(message1.getKey(), "key1");
+            Assert.assertEquals(new String(message1.getData()), "my-message-1");
+            Assert.assertEquals(message1.getMessageId(), messages.get(0).getMessageId());
+
+            Message message2 = consumer.receive();
+            Assert.assertEquals(message2.getKey(), "key2");
+            Assert.assertEquals(new String(message2.getData()), "my-message-3");
+            Assert.assertEquals(message2.getMessageId(), messages.get(2).getMessageId());
+        }
+    }
+
+    @Test
+    public void testWholeBatchCompactedOut() throws Exception {
+        String topic = "persistent://my-property/use/my-ns/my-topic1";
+
+        // subscribe before sending anything, so that we get all messages
+        pulsarClient.newConsumer().topic(topic).subscriptionName("sub1")
+            .readCompacted(true).subscribe().close();
+
+        try (Producer producerNormal = pulsarClient.newProducer().topic(topic).create();
+             Producer producerBatch = pulsarClient.newProducer().topic(topic).maxPendingMessages(3)
+                .enableBatching(true).batchingMaxMessages(3)
+                .batchingMaxPublishDelay(1, TimeUnit.HOURS).create()) {
+            producerBatch.sendAsync(MessageBuilder.create()
+                                    .setKey("key1")
+                                    .setContent("my-message-1".getBytes()).build());
+            producerBatch.sendAsync(MessageBuilder.create()
+                                    .setKey("key1")
+                                    .setContent("my-message-2".getBytes()).build());
+            producerBatch.sendAsync(MessageBuilder.create()
+                                    .setKey("key1")
+                                    .setContent("my-message-3".getBytes()).build()).get();
+            producerNormal.sendAsync(MessageBuilder.create()
+                                     .setKey("key1")
+                                     .setContent("my-message-4".getBytes()).build()).get();
+        }
+
+        // compact the topic
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler);
+        compactor.compact(topic).get();
+
+        try (Consumer consumer = pulsarClient.newConsumer().topic(topic)
+                .subscriptionName("sub1").readCompacted(true).subscribe()){
+            Message message = consumer.receive();
+            Assert.assertEquals(message.getKey(), "key1");
+            Assert.assertEquals(new String(message.getData()), "my-message-4");
+        }
+    }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services