forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[feat][broker][PIP-278] Support pluggable topic compaction service - …
…part1 (apache#20645)
- Loading branch information
Showing
12 changed files
with
449 additions
and
13 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
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
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
48 changes: 48 additions & 0 deletions
48
pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactionServiceFactory.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 | 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); | ||
} |
84 changes: 84 additions & 0 deletions
84
pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarCompactionServiceFactory.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 | 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 | ||
} | ||
} |
111 changes: 111 additions & 0 deletions
111
pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarTopicCompactionService.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 | 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; | ||
} | ||
} |
Oops, something went wrong.