Skip to content

[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
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
94 changes: 94 additions & 0 deletions pip/pip-387.md
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**
Copy link
Contributor

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 the TopicCompactionService?

```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.
Copy link
Contributor

@liangyepianzhou liangyepianzhou Oct 29, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems to be a break change.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's acceptable in a new release. You can see TopicCompactionService is annotated with @InterfaceStability.Evolving.

  /**
   * 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: