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/01/11 01:13:06 UTC

[GitHub] rdhabalia commented on a change in pull request #1044: Compact algo

rdhabalia commented on a change in pull request #1044: Compact algo
URL: https://github.com/apache/incubator-pulsar/pull/1044#discussion_r160842871
 
 

 ##########
 File path: pulsar-broker/src/main/java/org/apache/pulsar/compaction/Compactor.java
 ##########
 @@ -0,0 +1,205 @@
+/**
+ * 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.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+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.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;
+
+/**
+ * Compactor for Pulsar topics
+ *
+ * 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 Compactor {
+    private static final Logger log = LoggerFactory.getLogger(Compactor.class);
+    private static final String COMPACTION_SUBSCRIPTION = "__compaction";
+    private static final int MAX_OUTSTANDING_READS = 500;
+    private static final int COMPACTED_TOPIC_ENSEMBLE_SIZE = 3;
+    private static final int COMPACTED_TOPIC_ACK_QUORUM_SIZE = 3;
+    private static final String COMPACTED_TOPIC_LEDGER_PROPERTY = "CompactedTopicLedger";
+    static byte[] COMPACTED_TOPIC_LEDGER_PASSWORD = COMPACTION_SUBSCRIPTION.getBytes();
+
+    private final ExecutorService executor;
+    private final PulsarClient pulsar;
+    private final BookKeeper bk;
+
+    public Compactor(PulsarClient pulsar,
+                     BookKeeper bk,
+                     ExecutorService executor) {
+        this.executor = executor;
+        this.pulsar = pulsar;
+        this.bk = bk;
+    }
+
+    public CompletableFuture<Long> compact(String topic) {
+        final CompletableFuture<Long> future
+            = new CompletableFuture<>();
+        executor.submit(new CompactionTask(topic, future));
+        return future;
+    }
+
+    private class CompactionTask implements Runnable {
+        final String topic;
+        final CompletableFuture<Long> future;
+
+        CompactionTask(String topic, CompletableFuture<Long> future) {
+            this.topic = topic;
+            this.future = future;
+        }
+
+        @Override
+        public void run() {
+            RawReader reader = null;
+            try {
+                reader = RawReader.create(pulsar, topic,
+                                          COMPACTION_SUBSCRIPTION).get(10, TimeUnit.SECONDS);
+                Map<String,MessageId> latestForKey = new HashMap<>();
+
+                RawMessage firstMessage = reader.readNextAsync().get(10, TimeUnit.SECONDS);
+                MessageId firstMessageId = firstMessage.getMessageId();
+                latestForKey.put(extractKey(firstMessage), firstMessageId);
+                MessageId latestMessageId = firstMessageId;
+
+                BlockingQueue<Future<RawMessage>> futures = new ArrayBlockingQueue<Future<RawMessage>>(
+                        MAX_OUTSTANDING_READS);
+                try {
+                    while (true) { // breaks with exception when there's no more messages
+                        Future<RawMessage> f = reader.readNextAsync();
+                        if (!futures.offer(f)) {
+                            RawMessage m = futures.take().get(10, TimeUnit.SECONDS);
+                            MessageId id = m.getMessageId();
+                            latestMessageId = id;
+                            String key = extractKey(m);
+                            latestForKey.put(key, id);
+                            futures.offer(f);
+                        }
+                    }
+                } catch (TimeoutException te) {
+                    // Can't read any more
+                }
+
+                futures.forEach(Compactor::cancelOrFree);
+                futures.clear();
+
+                reader.seekAsync(firstMessageId).get(10, TimeUnit.SECONDS);
+
+                LedgerHandle lh = bk.createLedger(COMPACTED_TOPIC_ENSEMBLE_SIZE,
+                                                  COMPACTED_TOPIC_ACK_QUORUM_SIZE,
+                                                  BookKeeper.DigestType.CRC32,
+                                                  COMPACTED_TOPIC_LEDGER_PASSWORD);
+                CompletableFuture<Void> lastAddFuture = CompletableFuture.completedFuture(null);
+                while (true) { // will break when it gets message matching latestMessageId
+                    Future<RawMessage> f = reader.readNextAsync();
+                    if (!futures.offer(f)) {
+                        RawMessage m = futures.take().get(10, TimeUnit.SECONDS);
+                        MessageId id = m.getMessageId();
+                        String key = extractKey(m);
+
+                        if (latestForKey.get(key).equals(id)) {
+                            final 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();
+                            lastAddFuture = bkf;
+                        }
+
+                        if (id.equals(latestMessageId)) {
+                            cancelOrFree(f);
+                            break;
+                        } else {
+                            futures.offer(f);
+                        }
+                    }
+                }
+                futures.forEach(Compactor::cancelOrFree);
+                futures.clear();
+
+                lastAddFuture.get(); // ensure all adds have completed
+                lh.close();
+
+                reader.acknowledgeCumulativeAsync(latestMessageId,
+                                                  ImmutableMap.of(COMPACTED_TOPIC_LEDGER_PROPERTY,
+                                                                  lh.getId())).get();
+                future.complete(lh.getId());
+            } catch (Exception e) {
+                future.completeExceptionally(e);
+            } finally {
+                if (reader != null) {
+                    reader.closeAsync();
+                }
+            }
+        }
+    }
+
+    private static void cancelOrFree(Future<RawMessage> f) {
+        if (!f.cancel(false)) {
+            try {
+                f.get().close();
+            } catch (InterruptedException ie) {
+                Thread.currentThread().interrupt();
+            } catch (ExecutionException e) {
+                // we don't care, we're trying to cancel anyhow
+            }
+        }
+    }
+
+    private static String extractKey(RawMessage m) throws Exception {
+        ByteBuf headersAndPayload = m.getHeadersAndPayload();
+        MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload);
+        return msgMetadata.getPartitionKey();
 
 Review comment:
   If key would be always partition-key then does it mean that compaction keeps only 1 latest message for each individual partitioned topic? 
   

----------------------------------------------------------------
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