merlimat closed pull request #1044: Compact algo
URL: https://github.com/apache/incubator-pulsar/pull/1044
 
 
   

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/client/api/RawMessage.java 
b/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawMessage.java
index cc983e823..e6d3081b4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawMessage.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawMessage.java
@@ -46,10 +46,10 @@
     ByteBuf getHeadersAndPayload();
 
     /**
-     * Serialize a raw message to a ByteBufPair. The caller is responsible for 
releasing
-     * the returned ByteBufPair.
+     * Serialize a raw message to a ByteBuf. The caller is responsible for 
releasing
+     * the returned ByteBuf.
      */
-    ByteBufPair serialize();
+    ByteBuf serialize();
 
     @Override
     void close();
diff --git 
a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawMessageImpl.java 
b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawMessageImpl.java
index 7160cc3be..d1b6e14bc 100644
--- 
a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawMessageImpl.java
+++ 
b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawMessageImpl.java
@@ -65,15 +65,18 @@ public void close() {
     }
 
     @Override
-    public ByteBufPair serialize() {
+    public ByteBuf serialize() {
+        ByteBuf headersAndPayload = this.headersAndPayload.slice();
+
         // Format: [IdSize][Id][PayloadAndMetadataSize][PayloadAndMetadata]
         int idSize = id.getSerializedSize();
         int headerSize = 4 /* IdSize */ + idSize + 4 /* PayloadAndMetadataSize 
*/;
+        int totalSize = headerSize + headersAndPayload.readableBytes();
 
-        ByteBuf headers = PooledByteBufAllocator.DEFAULT.buffer(headerSize);
-        headers.writeInt(idSize);
+        ByteBuf buf = PooledByteBufAllocator.DEFAULT.buffer(totalSize);
+        buf.writeInt(idSize);
         try {
-            ByteBufCodedOutputStream outStream = 
ByteBufCodedOutputStream.get(headers);
+            ByteBufCodedOutputStream outStream = 
ByteBufCodedOutputStream.get(buf);
             id.writeTo(outStream);
             outStream.recycle();
         } catch (IOException e) {
@@ -81,9 +84,10 @@ public ByteBufPair serialize() {
             log.error("IO exception serializing to ByteBuf (this shouldn't 
happen as operation is in-memory)", e);
             throw new RuntimeException(e);
         }
-        headers.writeInt(headersAndPayload.readableBytes());
+        buf.writeInt(headersAndPayload.readableBytes());
+        buf.writeBytes(headersAndPayload);
 
-        return ByteBufPair.get(headers, headersAndPayload);
+        return buf;
     }
 
     static public RawMessage deserializeFrom(ByteBuf buffer) {
diff --git 
a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/Compactor.java 
b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/Compactor.java
new file mode 100644
index 000000000..62e5fcdee
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/Compactor.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.compaction;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+
+import org.apache.bookkeeper.client.BookKeeper;
+
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.RawReader;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Compactor for Pulsar topics
+*/
+public abstract class Compactor {
+    private static final Logger log = LoggerFactory.getLogger(Compactor.class);
+    private static final String COMPACTION_SUBSCRIPTION = "__compaction";
+    private static final String COMPACTED_TOPIC_LEDGER_PROPERTY = 
"CompactedTopicLedger";
+    static BookKeeper.DigestType COMPACTED_TOPIC_LEDGER_DIGEST_TYPE = 
BookKeeper.DigestType.CRC32;
+    static byte[] COMPACTED_TOPIC_LEDGER_PASSWORD = "".getBytes(UTF_8);
+
+    protected final ServiceConfiguration conf;
+    protected final ScheduledExecutorService scheduler;
+    private final PulsarClient pulsar;
+    private final BookKeeper bk;
+
+    public Compactor(ServiceConfiguration conf,
+                     PulsarClient pulsar,
+                     BookKeeper bk,
+                     ScheduledExecutorService scheduler) {
+        this.conf = conf;
+        this.scheduler = scheduler;
+        this.pulsar = pulsar;
+        this.bk = bk;
+    }
+
+    public CompletableFuture<Long> compact(String topic) {
+        return RawReader.create(pulsar, topic, 
COMPACTION_SUBSCRIPTION).thenComposeAsync(
+                this::compactAndCloseReader, scheduler);
+    }
+
+    private CompletableFuture<Long> compactAndCloseReader(RawReader reader) {
+        CompletableFuture<Long> promise = new CompletableFuture<>();
+        doCompaction(reader, bk).whenComplete(
+                (ledgerId, exception) -> {
+                    reader.closeAsync().whenComplete((v, exception2) -> {
+                            if (exception2 != null) {
+                                log.warn("Error closing reader handle {}, 
ignoring", reader, exception2);
+                            }
+                            if (exception != null) {
+                                // complete with original exception
+                                promise.completeExceptionally(exception);
+                            } else {
+                                promise.complete(ledgerId);
+                            }
+                        });
+                });
+        return promise;
+    }
+
+    protected abstract CompletableFuture<Long> doCompaction(RawReader reader, 
BookKeeper bk);
+}
+
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
new file mode 100644
index 000000000..957e1970c
--- /dev/null
+++ 
b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
@@ -0,0 +1,285 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.compaction;
+
+import com.google.common.collect.ImmutableMap;
+import io.netty.buffer.ByteBuf;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.common.api.Commands;
+import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.RawReader;
+import org.apache.pulsar.client.api.RawMessage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Compaction will go through the topic in two passes. The first pass
+ * selects latest offset for each key in the topic. Then the second pass
+ * writes these values to a ledger.
+ *
+ * <p>The two passes are required to avoid holding the payloads of each of
+ * the latest values in memory, as the payload can be many orders of
+ * magnitude larger than a message id.
+*/
+public class TwoPhaseCompactor extends Compactor {
+    private static final Logger log = 
LoggerFactory.getLogger(TwoPhaseCompactor.class);
+    private static final int MAX_OUTSTANDING = 500;
+    private static final String COMPACTED_TOPIC_LEDGER_PROPERTY = 
"CompactedTopicLedger";
+
+    public TwoPhaseCompactor(ServiceConfiguration conf,
+                             PulsarClient pulsar,
+                             BookKeeper bk,
+                             ScheduledExecutorService scheduler) {
+        super(conf, pulsar, bk, scheduler);
+    }
+
+    @Override
+    protected CompletableFuture<Long> doCompaction(RawReader reader, 
BookKeeper bk) {
+        return phaseOne(reader).thenCompose(
+                (r) -> phaseTwo(reader, r.from, r.to, r.latestForKey, bk));
+    }
+
+    private CompletableFuture<PhaseOneResult> phaseOne(RawReader reader) {
+        Map<String,MessageId> latestForKey = new HashMap<>();
+
+        CompletableFuture<PhaseOneResult> loopPromise = new 
CompletableFuture<>();
+        phaseOneLoop(reader, Optional.empty(), Optional.empty(), latestForKey, 
loopPromise);
+        return loopPromise;
+    }
+
+    private void phaseOneLoop(RawReader reader,
+                              Optional<MessageId> firstMessageId,
+                              Optional<MessageId> lastMessageId,
+                              Map<String,MessageId> latestForKey,
+                              CompletableFuture<PhaseOneResult> loopPromise) {
+        if (loopPromise.isDone()) {
+            return;
+        }
+        CompletableFuture<RawMessage> future = reader.readNextAsync();
+        scheduleTimeout(future);
+        future.whenComplete(
+                (m, exception) -> {
+                    try {
+                        if (exception != null) {
+                            if (exception instanceof TimeoutException
+                                && firstMessageId.isPresent()) {
+                                loopPromise.complete(new 
PhaseOneResult(firstMessageId.get(),
+                                                                        
lastMessageId.get(),
+                                                                        
latestForKey));
+                            } else {
+                                loopPromise.completeExceptionally(exception);
+                            }
+                            return;
+                        }
+
+                        MessageId id = m.getMessageId();
+                        String key = extractKey(m);
+                        latestForKey.put(key, id);
+
+                        phaseOneLoop(reader,
+                                     Optional.of(firstMessageId.orElse(id)),
+                                     Optional.of(id),
+                                     latestForKey, loopPromise);
+                    } finally {
+                        m.close();
+                    }
+                });
+
+    }
+
+    private void scheduleTimeout(CompletableFuture<RawMessage> future) {
+        Future<?> timeout = scheduler.schedule(() -> {
+                future.completeExceptionally(new TimeoutException("Timeout"));
+            }, 10, TimeUnit.SECONDS);
+        future.whenComplete((res, exception) -> {
+                timeout.cancel(true);
+            });
+    }
+
+    private CompletableFuture<Long> phaseTwo(RawReader reader, MessageId from, 
MessageId to,
+                                             Map<String,MessageId> 
latestForKey, BookKeeper bk) {
+        return createLedger(bk).thenCompose(
+                (ledger) -> phaseTwoSeekThenLoop(reader, from, to, 
latestForKey, bk, ledger));
+    }
+
+    private CompletableFuture<Long> phaseTwoSeekThenLoop(RawReader reader, 
MessageId from, MessageId to,
+                                                         Map<String, 
MessageId> latestForKey,
+                                                         BookKeeper bk, 
LedgerHandle ledger) {
+        CompletableFuture<Long> promise = new CompletableFuture<>();
+
+        reader.seekAsync(from).thenCompose((v) -> {
+                Semaphore outstanding = new Semaphore(MAX_OUTSTANDING);
+                CompletableFuture<Void> loopPromise = new 
CompletableFuture<Void>();
+                phaseTwoLoop(reader, to, latestForKey, ledger, outstanding, 
loopPromise);
+                return loopPromise;
+            }).thenCompose((v) -> closeLedger(ledger))
+            .thenCompose((v) -> reader.acknowledgeCumulativeAsync(
+                                 to, 
ImmutableMap.of(COMPACTED_TOPIC_LEDGER_PROPERTY, ledger.getId())))
+            .whenComplete((res, exception) -> {
+                    if (exception != null) {
+                        deleteLedger(bk, ledger)
+                            .whenComplete((res2, exception2) -> {
+                                    if (exception2 != null) {
+                                        log.warn("Cleanup of ledger {} for 
failed", ledger, exception2);
+                                    }
+                                    // complete with original exception
+                                    promise.completeExceptionally(exception);
+                                });
+                    } else {
+                        promise.complete(ledger.getId());
+                    }
+                });
+        return promise;
+    }
+
+    private void phaseTwoLoop(RawReader reader, MessageId to, Map<String, 
MessageId> latestForKey,
+                              LedgerHandle lh, Semaphore outstanding, 
CompletableFuture<Void> promise) {
+        reader.readNextAsync().whenComplete(
+                (m, exception) -> {
+                    try {
+                        if (exception != null) {
+                            promise.completeExceptionally(exception);
+                            return;
+                        } else if (promise.isDone()) {
+                            return;
+                        }
+                        MessageId id = m.getMessageId();
+                        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);
+                                        }
+                                    });
+                            }
+                        }
+                        phaseTwoLoop(reader, to, latestForKey, lh, 
outstanding, promise);
+                    } catch (InterruptedException ie) {
+                        Thread.currentThread().interrupt();
+                        promise.completeExceptionally(ie);
+                    } finally {
+                        m.close();
+                    }
+                });
+    }
+
+    private CompletableFuture<LedgerHandle> createLedger(BookKeeper bk) {
+        CompletableFuture<LedgerHandle> bkf = new CompletableFuture<>();
+        bk.asyncCreateLedger(conf.getManagedLedgerDefaultEnsembleSize(),
+                             conf.getManagedLedgerDefaultWriteQuorum(),
+                             conf.getManagedLedgerDefaultAckQuorum(),
+                             Compactor.COMPACTED_TOPIC_LEDGER_DIGEST_TYPE,
+                             Compactor.COMPACTED_TOPIC_LEDGER_PASSWORD,
+                             (rc, ledger, ctx) -> {
+                                 if (rc != BKException.Code.OK) {
+                                     
bkf.completeExceptionally(BKException.create(rc));
+                                 } else {
+                                     bkf.complete(ledger);
+                                 }
+                             }, null);
+        return bkf;
+    }
+
+    private CompletableFuture<Void> deleteLedger(BookKeeper bk, LedgerHandle 
lh) {
+        CompletableFuture<Void> bkf = new CompletableFuture<>();
+        bk.asyncDeleteLedger(lh.getId(),
+                             (rc, ctx) -> {
+                                 if (rc != BKException.Code.OK) {
+                                     
bkf.completeExceptionally(BKException.create(rc));
+                                 } else {
+                                     bkf.complete(null);
+                                 }
+                             }, null);
+        return bkf;
+    }
+
+    private CompletableFuture<Void> closeLedger(LedgerHandle lh) {
+        CompletableFuture<Void> bkf = new CompletableFuture<>();
+        lh.asyncClose((rc, ledger, ctx) -> {
+                if (rc != BKException.Code.OK) {
+                    bkf.completeExceptionally(BKException.create(rc));
+                } else {
+                    bkf.complete(null);
+                }
+            }, null);
+        return bkf;
+    }
+
+    private CompletableFuture<Void> addToCompactedLedger(LedgerHandle lh, 
RawMessage m) {
+        CompletableFuture<Void> bkf = new CompletableFuture<>();
+        ByteBuf serialized = m.serialize();
+        lh.asyncAddEntry(serialized,
+                         (rc, ledger, eid, ctx) -> {
+                             if (rc != BKException.Code.OK) {
+                                 
bkf.completeExceptionally(BKException.create(rc));
+                             } else {
+                                 bkf.complete(null);
+                             }
+                         }, null);
+        serialized.release();
+        return bkf;
+    }
+
+    private static String extractKey(RawMessage m) {
+        ByteBuf headersAndPayload = m.getHeadersAndPayload();
+        MessageMetadata msgMetadata = 
Commands.parseMessageMetadata(headersAndPayload);
+        return msgMetadata.getPartitionKey();
+    }
+
+    private static class PhaseOneResult {
+        final MessageId from;
+        final MessageId to;
+        final Map<String,MessageId> latestForKey;
+
+        PhaseOneResult(MessageId from, MessageId to, Map<String,MessageId> 
latestForKey) {
+            this.from = from;
+            this.to = to;
+            this.latestForKey = latestForKey;
+        }
+    }
+}
diff --git 
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawMessageSerDeserTest.java
 
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawMessageSerDeserTest.java
index 035bbf973..79ed79465 100644
--- 
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawMessageSerDeserTest.java
+++ 
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawMessageSerDeserTest.java
@@ -44,7 +44,7 @@ public void testSerializationAndDeserialization() throws 
Exception {
             .setPartition(10).setBatchIndex(20).build();
 
         RawMessage m = new RawMessageImpl(id, headersAndPayload);
-        ByteBuf serialized = ByteBufPair.coalesce(m.serialize());
+        ByteBuf serialized = m.serialize();
         byte[] bytes = new byte[serialized.readableBytes()];
         serialized.readBytes(bytes);
 
diff --git 
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java 
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java
index cef3e428a..19cdc3155 100644
--- 
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java
+++ 
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java
@@ -116,7 +116,7 @@ public void cleanup() throws Exception {
         return publishMessagesBase(topic, count, true);
     }
 
-    private static String extractKey(RawMessage m) throws Exception {
+    public static String extractKey(RawMessage m) throws Exception {
         ByteBuf headersAndPayload = m.getHeadersAndPayload();
         MessageMetadata msgMetadata = 
Commands.parseMessageMetadata(headersAndPayload);
         return msgMetadata.getPartitionKey();
diff --git 
a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java 
b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java
new file mode 100644
index 000000000..b517a90c8
--- /dev/null
+++ 
b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java
@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.compaction;
+
+import static org.apache.pulsar.client.impl.RawReaderTest.extractKey;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.buffer.ByteBuf;
+
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.common.api.Commands;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.PropertyAdmin;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConfiguration;
+import org.apache.pulsar.client.api.MessageBuilder;
+import org.apache.pulsar.client.api.RawMessage;
+import org.apache.pulsar.client.impl.RawMessageImpl;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+public class CompactorTest extends MockedPulsarServiceBaseTest {
+    private static final Logger log = 
LoggerFactory.getLogger(CompactorTest.class);
+
+    private ScheduledExecutorService compactionScheduler;
+
+    @BeforeMethod
+    @Override
+    public void setup() throws Exception {
+        super.internalSetup();
+
+        admin.clusters().createCluster("use",
+                new ClusterData("http://127.0.0.1:"; + BROKER_WEBSERVICE_PORT));
+        admin.properties().createProperty("my-property",
+                new PropertyAdmin(Lists.newArrayList("appid1", "appid2"), 
Sets.newHashSet("use")));
+        admin.namespaces().createNamespace("my-property/use/my-ns");
+
+        compactionScheduler = Executors.newSingleThreadScheduledExecutor(
+                new 
ThreadFactoryBuilder().setNameFormat("compactor").setDaemon(true).build());
+    }
+
+    @AfterMethod
+    @Override
+    public void cleanup() throws Exception {
+        super.internalCleanup();
+
+        compactionScheduler.shutdownNow();
+    }
+
+    private List<String> compactAndVerify(String topic, Map<String, byte[]> 
expected) throws Exception {
+        BookKeeper bk = pulsar.getBookKeeperClientFactory().create(
+                this.conf, null);
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, 
compactionScheduler);
+        long compactedLedgerId = compactor.compact(topic).get();
+
+        LedgerHandle ledger = bk.openLedger(compactedLedgerId,
+                                            
Compactor.COMPACTED_TOPIC_LEDGER_DIGEST_TYPE,
+                                            
Compactor.COMPACTED_TOPIC_LEDGER_PASSWORD);
+        Assert.assertEquals(ledger.getLastAddConfirmed() + 1, // 0..lac
+                            expected.size(),
+                            "Should have as many entries as there is keys");
+
+        List<String> keys = new ArrayList<>();
+        Enumeration<LedgerEntry> entries = ledger.readEntries(0, 
ledger.getLastAddConfirmed());
+        while (entries.hasMoreElements()) {
+            ByteBuf buf = entries.nextElement().getEntryBuffer();
+            RawMessage m = RawMessageImpl.deserializeFrom(buf);
+            String key = extractKey(m);
+            keys.add(key);
+
+            ByteBuf payload = extractPayload(m);
+            byte[] bytes = new byte[payload.readableBytes()];
+            payload.readBytes(bytes);
+            Assert.assertEquals(bytes, expected.remove(key),
+                                "Compacted version should match expected 
version");
+            m.close();
+        }
+        Assert.assertTrue(expected.isEmpty(), "All expected keys should have 
been found");
+        return keys;
+    }
+
+    @Test
+    public void testCompaction() throws Exception {
+        String topic = "persistent://my-property/use/my-ns/my-topic1";
+        final int numMessages = 1000;
+        final int maxKeys = 10;
+
+        ProducerConfiguration producerConf = new ProducerConfiguration();
+        Producer producer = pulsarClient.createProducer(topic, producerConf);
+
+        Map<String, byte[]> expected = new HashMap<>();
+        Random r = new Random(0);
+
+        for (int j = 0; j < numMessages; j++) {
+            int keyIndex = r.nextInt(maxKeys);
+            String key = "key"+keyIndex;
+            byte[] data = ("my-message-" + key + "-" + j).getBytes();
+            producer.send(MessageBuilder.create()
+                          .setKey(key)
+                          .setContent(data).build());
+            expected.put(key, data);
+        }
+        compactAndVerify(topic, expected);
+    }
+
+    @Test
+    public void testCompactAddCompact() throws Exception {
+        String topic = "persistent://my-property/use/my-ns/my-topic1";
+
+        ProducerConfiguration producerConf = new ProducerConfiguration();
+        Producer producer = pulsarClient.createProducer(topic, producerConf);
+
+        Map<String, byte[]> expected = new HashMap<>();
+
+        producer.send(MessageBuilder.create()
+                      .setKey("a")
+                      .setContent("A_1".getBytes()).build());
+        producer.send(MessageBuilder.create()
+                      .setKey("b")
+                      .setContent("B_1".getBytes()).build());
+        producer.send(MessageBuilder.create()
+                      .setKey("a")
+                      .setContent("A_2".getBytes()).build());
+        expected.put("a", "A_2".getBytes());
+        expected.put("b", "B_1".getBytes());
+
+        compactAndVerify(topic, new HashMap<>(expected));
+
+        producer.send(MessageBuilder.create()
+                      .setKey("b")
+                      .setContent("B_2".getBytes()).build());
+        expected.put("b", "B_2".getBytes());
+
+        compactAndVerify(topic, expected);
+    }
+
+    @Test
+    public void testCompactedInOrder() throws Exception {
+        String topic = "persistent://my-property/use/my-ns/my-topic1";
+
+        ProducerConfiguration producerConf = new ProducerConfiguration();
+        Producer producer = pulsarClient.createProducer(topic, producerConf);
+
+        producer.send(MessageBuilder.create()
+                      .setKey("c")
+                      .setContent("C_1".getBytes()).build());
+        producer.send(MessageBuilder.create()
+                      .setKey("a")
+                      .setContent("A_1".getBytes()).build());
+        producer.send(MessageBuilder.create()
+                      .setKey("b")
+                      .setContent("B_1".getBytes()).build());
+        producer.send(MessageBuilder.create()
+                      .setKey("a")
+                      .setContent("A_2".getBytes()).build());
+        Map<String, byte[]> expected = new HashMap<>();
+        expected.put("a", "A_2".getBytes());
+        expected.put("b", "B_1".getBytes());
+        expected.put("c", "C_1".getBytes());
+
+        List<String> keyOrder = compactAndVerify(topic, expected);
+
+        Assert.assertEquals(keyOrder, Lists.newArrayList("c", "b", "a"));
+    }
+
+    @Test(expectedExceptions = ExecutionException.class)
+    public void testCompactEmptyTopic() throws Exception {
+        String topic = "persistent://my-property/use/my-ns/my-topic1";
+
+        // trigger creation of topic on server side
+        pulsarClient.subscribe(topic, "sub1").close();
+
+        BookKeeper bk = pulsar.getBookKeeperClientFactory().create(
+                this.conf, null);
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, 
compactionScheduler);
+        compactor.compact(topic).get();
+    }
+
+    public ByteBuf extractPayload(RawMessage m) throws Exception {
+        ByteBuf payloadAndMetadata = m.getHeadersAndPayload();
+        Commands.readChecksum(payloadAndMetadata);
+        int metadataSize = payloadAndMetadata.readInt(); // metadata size
+         byte[] metadata = new byte[metadataSize];
+        payloadAndMetadata.readBytes(metadata);
+        return payloadAndMetadata.slice();
+    }
+}


 

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to