-
Notifications
You must be signed in to change notification settings - Fork 3.6k
[improve] [pip] PIP-387: Modify interface TopicCompactionService to support cancelling an in-progress compaction task #23462
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
Open
poorbarcode
wants to merge
2
commits into
apache:master
Choose a base branch
from
poorbarcode:pip/387
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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,94 @@ | ||
# PIP-387: Modify interface TopicCompactionService to support cancelling an in-progress compaction task | ||
|
||
# Background knowledge | ||
|
||
- What is Topic Compaction: https://pulsar.apache.org/docs/3.3.x/cookbooks-compaction | ||
- How it works: it reads messages from the target topic, and compacts them into a single ledger, which will be stored with cursor's properties. | ||
|
||
# Motivation | ||
|
||
- There is a bug that may lead a deadlock when deleting topic and topic compaction execute at the same time. | ||
|
||
| time | `deleting topic` | `compaction` | | ||
| --- | --- | ---| | ||
| 1 | | start to create the internal reader | | ||
| 2 | mark the topic is deleting, and the deleting has not finished yet | | | ||
| 3 | disconnect all client, which includes the internal reader for compaction| | | ||
| 4 | | loop to reconnect | | ||
| 5 | waiting for in-progress compaction, to delete the ledger that contains compacted messages | | | ||
| 6 | keep waiting... | keeping reconecting | | ||
|
||
- And, there is another situation which is weird when unloading topic and compaction execute at the same time, and the result of this scenario is okay, because the fist compaction task will eventually succeed. | ||
|
||
| time | `broker-1` | `broker-2` | | ||
| --- | --- | ---| | ||
| 1 | starts compaction | | | ||
| 2 | unload topic due to a load-balance | | ||
| 3 | close the topic | | ||
| 4 | the compaction is still running | | ||
| 5 | | load up the topic | | ||
| 6 | | starts compaction | | ||
| 7 | | receive an error: "Exclusive consumer is already connected"| | ||
|
||
# Goals | ||
|
||
Modify interface `TopicCompactionService` to support cancelling an in-progress compaction task | ||
|
||
# API changes | ||
|
||
**TopicCompactionService.java** | ||
```java | ||
- CompletableFuture<Void> compact(); | ||
+ CompactionTaskCtx compact(); | ||
``` | ||
|
||
**CompactionTaskCtx.java** | ||
```java | ||
// Used to cancel the in-progress compaction task, in other words, close the reader immediately. | ||
private Reader reader; | ||
// Used to print logs. | ||
private String topic; | ||
// Compaction task | ||
private final CompletableFuture<Long> task; | ||
// State of the task. | ||
private volatile State state; | ||
``` | ||
|
||
**CompactionTaskCtx.State.java(enum)** | ||
- `Creating_Raw_Reader`: the reader's subscribing has not finished yet. | ||
- `Running`: the task is running. | ||
- `Ending_with_switch_ledger`: the task is nearly finished, it is trying to update ledger info into the cursor's metadata. | ||
- End states | ||
- `Cancelled` | ||
- `Succeed` | ||
- `Failed` | ||
|
||
# Detailed Design | ||
- The state of `CompactionTaskCtx` will be modified multi times when a compaction task executes. | ||
- Broker will try to cancel the in-progress compaction task when it is deleting or closing. | ||
- `Ending_with_switch_ledger` is an important state, before the task switching to this state, the task can be cancelled. Otherwise, the task can not be cancelled, because we do not know whether updating cursor's metadata will finish or fail. | ||
- The implementation of`CompactionTaskCtx.cancel()`: broker will close the reader that used for compaction once it changed the state to `Cancelled`. The following error will occur after the reader is closed, and we can handle them internally. | ||
- `state: Creating_Raw_Reader`: the reconnection looping will stop automatically, and the future indicates subscribing will never complete. | ||
- `state: Running`: | ||
- `reader.getLastMessageIdAsync` will get a `PulsarClientException.TimeoutException` or `PulsarClientException.AlreadyClosedException` | ||
- `reader.readNextAsync` will get a `java.util.concurrent.CancellationException` or `PulsarClientException.AlreadyClosedException` | ||
- `reader.seekAsync` will get a `PulsarClientException.TimeoutException` or `PulsarClientException.AlreadyClosedException` | ||
- `reader.acknowledgeCumulativeAsync` will get a `PulsarClientException.TimeoutException`, `PulsarClientException.ConnectException` or `PulsarClientException.AlreadyClosedException` | ||
|
||
# Backward & Forward Compatibility | ||
|
||
## Upgrade | ||
|
||
If you did not implement a customized `TopicCompactionService` or `Compactor`, it is safety. Otherwise, you need to change your code. | ||
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 to be a break change. 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's acceptable in a new release. You can see /**
* Evolving, but can break compatibility at minor release (i.e. m.x)
*/
@Documented
public @interface Evolving {} |
||
|
||
## Downgrade / Rollback | ||
|
||
Nothing. | ||
|
||
# Links | ||
|
||
<!-- | ||
Updated afterwards | ||
--> | ||
* Mailing List discussion thread: https://lists.apache.org/thread/brh9816o0xnkkxqbpndj69c25opg39lo | ||
* Mailing List voting thread: |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
This class exposes too many details. Why not just add a
cancel()
to theTopicCompactionService
?