Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
* 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
Showing
7 changed files
with
609 additions
and
11 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
pulsar-broker/src/main/java/org/apache/pulsar/compaction/Compactor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -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); | |||
} | |||
|
Oops, something went wrong.