Skip to content

Commit

Permalink
Compact algo (#1044)
Browse files Browse the repository at this point in the history
* Compaction algorithm for Pulsar

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.

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.

* fix nits

* fixup for compactor, empty test

* Shade plugin doesn't like CompletableFuture<?>, which is weird

* Address @rdhabalia comments

* Fix stupid compile error
  • Loading branch information
ivankelly authored and merlimat committed Jan 29, 2018
1 parent b3fe055 commit a499850
Show file tree
Hide file tree
Showing 7 changed files with 609 additions and 11 deletions.
Expand Up @@ -46,10 +46,10 @@ public interface RawMessage extends AutoCloseable {
ByteBuf getHeadersAndPayload(); ByteBuf getHeadersAndPayload();


/** /**
* Serialize a raw message to a ByteBufPair. The caller is responsible for releasing * Serialize a raw message to a ByteBuf. The caller is responsible for releasing
* the returned ByteBufPair. * the returned ByteBuf.
*/ */
ByteBufPair serialize(); ByteBuf serialize();


@Override @Override
void close(); void close();
Expand Down
Expand Up @@ -65,25 +65,29 @@ public void close() {
} }


@Override @Override
public ByteBufPair serialize() { public ByteBuf serialize() {
ByteBuf headersAndPayload = this.headersAndPayload.slice();

// Format: [IdSize][Id][PayloadAndMetadataSize][PayloadAndMetadata] // Format: [IdSize][Id][PayloadAndMetadataSize][PayloadAndMetadata]
int idSize = id.getSerializedSize(); int idSize = id.getSerializedSize();
int headerSize = 4 /* IdSize */ + idSize + 4 /* PayloadAndMetadataSize */; int headerSize = 4 /* IdSize */ + idSize + 4 /* PayloadAndMetadataSize */;
int totalSize = headerSize + headersAndPayload.readableBytes();


ByteBuf headers = PooledByteBufAllocator.DEFAULT.buffer(headerSize); ByteBuf buf = PooledByteBufAllocator.DEFAULT.buffer(totalSize);
headers.writeInt(idSize); buf.writeInt(idSize);
try { try {
ByteBufCodedOutputStream outStream = ByteBufCodedOutputStream.get(headers); ByteBufCodedOutputStream outStream = ByteBufCodedOutputStream.get(buf);
id.writeTo(outStream); id.writeTo(outStream);
outStream.recycle(); outStream.recycle();
} catch (IOException e) { } catch (IOException e) {
// This is in-memory serialization, should not fail // This is in-memory serialization, should not fail
log.error("IO exception serializing to ByteBuf (this shouldn't happen as operation is in-memory)", e); log.error("IO exception serializing to ByteBuf (this shouldn't happen as operation is in-memory)", e);
throw new RuntimeException(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) { static public RawMessage deserializeFrom(ByteBuf buffer) {
Expand Down
@@ -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);
}

0 comments on commit a499850

Please sign in to comment.