Skip to content

Commit

Permalink
[feat][broker][PIP-278] Support pluggable topic compaction service - …
Browse files Browse the repository at this point in the history
…part1 (apache#20645)
  • Loading branch information
coderzc authored Jun 30, 2023
1 parent 6acd01d commit c82825b
Show file tree
Hide file tree
Showing 12 changed files with 449 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -3166,6 +3166,12 @@ The delayed message index time step(in seconds) in per bucket snapshot segment,
)
private int transactionPendingAckBatchedWriteMaxDelayInMillis = 1;

@FieldContext(
category = CATEGORY_SERVER,
doc = "The class name of the factory that implements the topic compaction service."
)
private String compactionServiceFactoryClassName = "org.apache.pulsar.compaction.PulsarCompactionServiceFactory";

/**** --- KeyStore TLS config variables. --- ****/
@FieldContext(
category = CATEGORY_KEYSTORE_TLS,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1470,7 +1470,7 @@ public BookKeeperClientFactory getBookKeeperClientFactory() {
return bkClientFactory;
}

protected synchronized ScheduledExecutorService getCompactorExecutor() {
public synchronized ScheduledExecutorService getCompactorExecutor() {
if (this.compactorExecutor == null) {
compactorExecutor = Executors.newSingleThreadScheduledExecutor(
new ExecutorProvider.ExtendedThreadFactory("compaction"));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,10 +37,10 @@
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.intercept.BrokerInterceptor;
import org.apache.pulsar.broker.service.persistent.CompactorSubscription;
import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.broker.service.persistent.PulsarCompactorSubscription;
import org.apache.pulsar.broker.service.plugin.EntryFilter;
import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl;
import org.apache.pulsar.client.api.transaction.TxnID;
Expand Down Expand Up @@ -301,7 +301,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i
}

private void individualAcknowledgeMessageIfNeeded(Position position, Map<String, Long> properties) {
if (!(subscription instanceof CompactorSubscription)) {
if (!(subscription instanceof PulsarCompactorSubscription)) {
subscription.acknowledgeMessage(Collections.singletonList(position), AckType.Individual, properties);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -483,7 +483,7 @@ private PersistentSubscription createPersistentSubscription(String subscriptionN
boolean replicated, Map<String, String> subscriptionProperties) {
Objects.requireNonNull(compactedTopic);
if (isCompactionSubscription(subscriptionName)) {
return new CompactorSubscription(this, compactedTopic, subscriptionName, cursor);
return new PulsarCompactorSubscription(this, compactedTopic, subscriptionName, cursor);
} else {
return new PersistentSubscription(this, subscriptionName, cursor, replicated, subscriptionProperties);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class CompactorSubscription extends PersistentSubscription {
public class PulsarCompactorSubscription extends PersistentSubscription {
private final CompactedTopic compactedTopic;

public CompactorSubscription(PersistentTopic topic, CompactedTopic compactedTopic,
String subscriptionName, ManagedCursor cursor) {
public PulsarCompactorSubscription(PersistentTopic topic, CompactedTopic compactedTopic,
String subscriptionName, ManagedCursor cursor) {
super(topic, subscriptionName, cursor, false);
checkArgument(subscriptionName.equals(Compactor.COMPACTION_SUBSCRIPTION));
this.compactedTopic = compactedTopic;
Expand Down Expand Up @@ -106,5 +106,5 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) {
}
}

private static final Logger log = LoggerFactory.getLogger(CompactorSubscription.class);
private static final Logger log = LoggerFactory.getLogger(PulsarCompactorSubscription.class);
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import javax.annotation.Nullable;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.LedgerEntry;
Expand Down Expand Up @@ -60,7 +61,7 @@ public class CompactedTopicImpl implements CompactedTopic {
private final BookKeeper bk;

private PositionImpl compactionHorizon = null;
private CompletableFuture<CompactedTopicContext> compactedTopicContext = null;
private volatile CompletableFuture<CompactedTopicContext> compactedTopicContext = null;

public CompactedTopicImpl(BookKeeper bk) {
this.bk = bk;
Expand Down Expand Up @@ -258,7 +259,7 @@ private static CompletableFuture<Void> tryDeleteCompactedLedger(BookKeeper bk, l
return promise;
}

private static CompletableFuture<List<Entry>> readEntries(LedgerHandle lh, long from, long to) {
static CompletableFuture<List<Entry>> readEntries(LedgerHandle lh, long from, long to) {
CompletableFuture<Enumeration<LedgerEntry>> promise = new CompletableFuture<>();

lh.asyncReadEntries(from, to,
Expand Down Expand Up @@ -320,6 +321,11 @@ private static int comparePositionAndMessageId(PositionImpl p, MessageIdData m)
public synchronized Optional<Position> getCompactionHorizon() {
return Optional.ofNullable(this.compactionHorizon);
}

@Nullable
public CompletableFuture<CompactedTopicContext> getCompactedTopicContextFuture() {
return compactedTopicContext;
}
private static final Logger log = LoggerFactory.getLogger(CompactedTopicImpl.class);
}

Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.annotations.Beta;
import java.util.concurrent.CompletableFuture;
import javax.annotation.Nonnull;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.common.classification.InterfaceAudience;

@Beta
@InterfaceAudience.Public
public interface CompactionServiceFactory extends AutoCloseable {

/**
* Initialize the compaction service factory.
*
* @param pulsarService
* the pulsar service instance
* @return a future represents the initialization result
*/
CompletableFuture<Void> initialize(@Nonnull PulsarService pulsarService);

/**
* Create a new topic compaction service for topic.
*
* @param topic
* the topic name
* @return a future represents the topic compaction service
*/
CompletableFuture<TopicCompactionService> newTopicCompactionService(@Nonnull String topic);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* 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.annotations.VisibleForTesting;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.PulsarService;

public class PulsarCompactionServiceFactory implements CompactionServiceFactory {

private PulsarService pulsarService;

private volatile Compactor compactor;

@VisibleForTesting
public Compactor getCompactor() throws PulsarServerException {
if (compactor == null) {
synchronized (this) {
if (compactor == null) {
compactor = newCompactor();
}
}
}
return compactor;
}

@Nullable
public Compactor getNullableCompactor() {
return compactor;
}

protected Compactor newCompactor() throws PulsarServerException {
return new TwoPhaseCompactor(pulsarService.getConfiguration(),
pulsarService.getClient(), pulsarService.getBookKeeperClient(),
pulsarService.getCompactorExecutor());
}

@Override
public CompletableFuture<Void> initialize(@Nonnull PulsarService pulsarService) {
Objects.requireNonNull(pulsarService);
this.pulsarService = pulsarService;
return CompletableFuture.completedFuture(null);
}

@Override
public CompletableFuture<TopicCompactionService> newTopicCompactionService(@Nonnull String topic) {
Objects.requireNonNull(topic);
PulsarTopicCompactionService pulsarTopicCompactionService =
new PulsarTopicCompactionService(topic, pulsarService.getBookKeeperClient(), () -> {
try {
return this.getCompactor();
} catch (Throwable e) {
throw new CompletionException(e);
}
});
return CompletableFuture.completedFuture(pulsarTopicCompactionService);
}

@Override
public void close() throws Exception {
// noop
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
/*
* 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 com.google.common.base.Preconditions.checkArgument;
import static org.apache.pulsar.compaction.CompactedTopicImpl.COMPACT_LEDGER_EMPTY;
import static org.apache.pulsar.compaction.CompactedTopicImpl.NEWER_THAN_COMPACTED;
import static org.apache.pulsar.compaction.CompactedTopicImpl.findStartPoint;
import java.util.Collections;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.function.Supplier;
import javax.annotation.Nonnull;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.pulsar.common.util.FutureUtil;


public class PulsarTopicCompactionService implements TopicCompactionService {

private final String topic;

private final CompactedTopicImpl compactedTopic;

private final Supplier<Compactor> compactorSupplier;

public PulsarTopicCompactionService(String topic, BookKeeper bookKeeper,
Supplier<Compactor> compactorSupplier) {
this.topic = topic;
this.compactedTopic = new CompactedTopicImpl(bookKeeper);
this.compactorSupplier = compactorSupplier;
}

@Override
public CompletableFuture<Void> compact() {
Compactor compactor;
try {
compactor = compactorSupplier.get();
} catch (Throwable e) {
return CompletableFuture.failedFuture(e);
}
return compactor.compact(topic).thenApply(x -> null);
}

@Override
public CompletableFuture<List<Entry>> readCompactedEntries(@Nonnull Position startPosition,
int numberOfEntriesToRead) {
Objects.requireNonNull(startPosition);
checkArgument(numberOfEntriesToRead > 0);

CompletableFuture<List<Entry>> resultFuture = new CompletableFuture<>();

Objects.requireNonNull(compactedTopic.getCompactedTopicContextFuture()).thenCompose(
(context) -> findStartPoint((PositionImpl) startPosition, context.ledger.getLastAddConfirmed(),
context.cache).thenCompose((startPoint) -> {
if (startPoint == COMPACT_LEDGER_EMPTY || startPoint == NEWER_THAN_COMPACTED) {
return CompletableFuture.completedFuture(Collections.emptyList());
}
long endPoint =
Math.min(context.ledger.getLastAddConfirmed(), startPoint + numberOfEntriesToRead);
return CompactedTopicImpl.readEntries(context.ledger, startPoint, endPoint);
})).whenComplete((result, ex) -> {
if (ex == null) {
resultFuture.complete(result);
} else {
ex = FutureUtil.unwrapCompletionException(ex);
if (ex instanceof NoSuchElementException) {
resultFuture.complete(Collections.emptyList());
} else {
resultFuture.completeExceptionally(ex);
}
}
});

return resultFuture;
}

@Override
public CompletableFuture<Entry> readLastCompactedEntry() {
return compactedTopic.readLastEntryOfCompactedLedger();
}

@Override
public CompletableFuture<Position> getLastCompactedPosition() {
return CompletableFuture.completedFuture(compactedTopic.getCompactionHorizon().orElse(null));
}

public CompactedTopicImpl getCompactedTopic() {
return compactedTopic;
}
}
Loading

0 comments on commit c82825b

Please sign in to comment.