-
Notifications
You must be signed in to change notification settings - Fork 3.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[pip][design] PIP-274: Support pluggable topic compactor #20493
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,126 @@ | ||
# Background knowledge | ||
|
||
Apache Pulsar is a distributed messaging system that supports multiple messaging protocols and storage methods. Among them, Pulsar Topic Compaction is a mechanism to clean up duplicate messages in topics to reduce storage space and improve system efficiency. | ||
More topic compaction details can be found in [Pulsar Topic Compaction](https://pulsar.apache.org/docs/en/concepts-topic-compaction/). | ||
|
||
# Motivation | ||
|
||
Currently, the implementation of Pulsar Topic Compaction is fixed and does not support custom strategy, which limits users from using more Compactor policies in their applications. | ||
|
||
|
||
For example, we need to parse the Kafka format then compact message in Kop, but the current implementation of Pulsar topic compaction does not support this feature. | ||
Another the topic compaction logic implemented in `TwoPhaseCompactor` only compacts messages to the last one, but sometimes we need to keep the first valid message e.g [`StrategicTwoPhaseCompactor`](https://github.com/coderzc/pulsar/blob/0e9935c493060b13b322a84c5418146423992369/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java). | ||
|
||
So we need to make the topic compactor pluggable to support more compaction strategy. | ||
|
||
# Goals | ||
|
||
## In Scope | ||
|
||
<!-- | ||
What this PIP intend to achieve once It's integrated into Pulsar. | ||
Why does it benefit Pulsar. | ||
--> | ||
|
||
Make the compactor pluggable. | ||
|
||
## Out of Scope | ||
|
||
<!-- | ||
Describe what you have decided to keep out of scope, perhaps left for a different PIP/s. | ||
--> | ||
|
||
|
||
# High Level Design | ||
|
||
<!-- | ||
Describe the design of your solution in *high level*. | ||
Describe the solution end to end, from a birds-eye view. | ||
Don't go into implementation details in this section. | ||
|
||
I should be able to finish reading from beginning of the PIP to here (including) and understand the feature and | ||
how you intend to solve it, end to end. | ||
|
||
DON'T | ||
* Avoid code snippets, unless it's essential to explain your intent. | ||
--> | ||
|
||
Make the topic compactor pluggable, users can customize the compactor implementation according to their own special scenarios. | ||
|
||
|
||
# Detailed Design | ||
|
||
## Design & Implementation Details | ||
|
||
<!-- | ||
This is the section where you dive into the details. It can be: | ||
* Concrete class names and their roles and responsibility, including methods. | ||
* Code snippets of existing code. | ||
* Interface names and its methods. | ||
* ... | ||
--> | ||
* Define a standard Compactor interface that specifies the methods and properties that the Compactor implementation needs to implement. This interface should include methods for Compactor initialization, Compactor execution, and getting Compactor stats. | ||
```java | ||
public interface Compactor { | ||
|
||
void initialize(ServiceConfiguration conf, | ||
PulsarClient pulsar, | ||
BookKeeper bk, | ||
ScheduledExecutorService scheduler); | ||
|
||
CompletableFuture<Long> compact(String topic); | ||
|
||
CompactorMXBean getStats(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think First, some metrics are going to be common for any compactor used:
Those metrics should be maintained by the service, which calls the compactor interface of I suggest using two interfaces which will be given in
There are some metrics I still need to figure out where to fit since I need to help understand something. The metrics are:
Once the compact() has ended, how does the compactor hand over the details on the output - may be the ledger ID, new compaction horizon, etc? I don't see that in the interface. Now, what's left is a way to record custom metrics specific to your compactor. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It seems using There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Extending is the worst. We should keep interfacing only. The interface creates a clean design that makes it easier to change in the future to some other mechanism (say OTel...). Somebody external to Pulsar needs to be able to depend as little as possible on Pulsar. Today adding metrics that are high cardinality (contain topic label) is done by writing them directly to SimpleTextOutputStream. You usually add a method for that, and you call it from the PrometheusMetricsGenerator. Thinking out loud - maybe we can add a method to
WDYT? |
||
} | ||
``` | ||
|
||
* Rename `org.apache.pulsar.compaction.Compactor` to `org.apache.pulsar.compaction.AbstractCompactor` and make it implement `Compactor` interface. | ||
|
||
* Load custom compactor based on configuration in `org.apache.pulsar.broker.PulsarService.newCompactor` and `CompactorTool`. | ||
|
||
## Public-facing Changes | ||
|
||
<!-- | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it's ok to remove all comments if you're done writing |
||
Describe the additions you plan to make for each public facing component. | ||
Remove the sections you are not changing. | ||
Clearly mark any changes which are BREAKING backward compatability. | ||
--> | ||
|
||
|
||
### Configuration | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
If I want There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do not support the namespace level configuration. This is a global configuration. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If I have a Pulsar cluster and there are some topics on this cluster, and we need to migrate Kafka to Pulsar now, how can I do? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Then, you copied the logic from the existing compactor? :( There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
For KafkaCompactor, you need to deal with Kafka Format and Pulsar Format, Kop can send Pulsar Format or Kafka Format There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Did you change the interface to support all functions of TwoPhaseCompactor? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Support all functions of TwoPhaseCompactor is not a required option, this is just a requirement in the case of Kop There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. oh There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I have three questions:
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Yes
Yes
If |
||
|
||
broker.conf | ||
``` | ||
compactorClassName=org.apache.pulsar.compaction.TwoPhaseCompactor | ||
``` | ||
|
||
# Backward & Forward Compatability | ||
|
||
## Revert | ||
|
||
<!-- | ||
Describe a cookbook detailing the steps required to revert pulsar to previous version *without* this feature. | ||
--> | ||
|
||
|
||
## Upgrade | ||
|
||
<!-- | ||
Specify the list of instructions, if there are such, needed to perform before/after upgrading to Pulsar version containing this feature. | ||
--> | ||
|
||
# Alternatives | ||
|
||
<!-- | ||
If there are alternatives that were already considered by the authors or, after the discussion, by the community, and were rejected, please list them here along with the reason why they were rejected. | ||
--> | ||
|
||
# General Notes | ||
|
||
# Links | ||
|
||
<!-- | ||
Updated afterwards | ||
--> | ||
* Mailing List discussion thread: https://lists.apache.org/thread/svyodx3q1nrdd6fyrsfv3xmo0lgn6j2j | ||
* Mailing List voting thread: |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's not a good abstraction to have
BookKeeper
(or evenPulsarClient
) involved. These parameters are just copied from the existingCompactor
's constructor, which is not abstracted well.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If the motivation is to handle messages whose format is other than pulsar, we might need to pass some functions like:
and pass the processor to the compactor.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@BewareMyPower Do you mean to abstract the
CompactEntryProcessor
and add acompactEntryProcessorClassName
configuration?But we still need to construct this
CompactEntryProcessor
by BookKeeper.How about abstraction the
CompactStorage
like:There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No. You understood wrong. Adding these two interfaces is complicated for users to implement. I mean, we only need ways to handle entries of different format, so users don't need to implement the same logic of reading entries or writing entries again.
For example, we only need to replace
with
Where the
processor
is an interface:Then, we don't need to rewrite the same logic again. We only need to change the logic about how to process a
RawMessage
.