Skip to content
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

Using Readers still causes backlog quota to be observed #6787

Merged
merged 1 commit into from
Jun 8, 2020

Conversation

jerrypeng
Copy link
Contributor

@jerrypeng jerrypeng commented Apr 21, 2020

Motivation

When using Readers API, backlog quotas are still enforced on these ephemeral readers.

Modifications

If a cursor is non-durable then we don't add it to the min-heap we are using keep track of the slowest cursor.

We use in the min-heap to access the slowest cursor for backlog and quota calculations.

It is also used to determine up to which ledger we can trim. Thus, with this change, reader's ephemeral subscriptions won't be preventing ledgers from being clean up as well which is what you expect the behavior to be.

There are some caveats to the above. Since triggering of trimming of ledgers is piggy packed on top of ledger operations, if there is no new data coming in ledgers will not be trimmed. The most recently closed ledger we be persisted past any retention. Perhaps in a subsequent PR, we implement the trimming to be trigger on a schedule as well.

Please note this PR includes changes from:

#6769

Verifying this change

Added a test to verify that readers don't cause any backlog issues

@jerrypeng jerrypeng self-assigned this Apr 21, 2020
@jerrypeng jerrypeng modified the milestone: 2.6.0 Apr 21, 2020
@jerrypeng jerrypeng added the type/bug The PR fixed a bug or issue reported a bug label Apr 21, 2020
@merlimat merlimat added this to the 2.6.0 milestone Apr 21, 2020
Copy link
Member

@sijie sijie left a comment

Choose a reason for hiding this comment

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

I'd like to understand the behavior described in the description - "It is also used to determine up to which ledger we can trim. Thus, with this change, reader's ephemeral subscriptions won't be preventing ledgers from being clean up as well which is what you expect the behavior to be."

Does this result in any data loss or unexpected behaviors when using the reader API?

@merlimat
Copy link
Contributor

Does this result in any data loss or unexpected behaviors when using the reader API?

The reader, per definition, doesn't cause the data to be retained. In this case, in the moment the reader is temporarily disconnected, the data would be deleted immediately, if the retention policy says so.

For that, an application using readers needs to configure appropriate retention.

The only difference here is when the readers are connected. The current behavior is to consider the reader subject to the backlog quota policy, which is incorrect given that backlog only applies to a subscription.

@jerrypeng
Copy link
Contributor Author

@sijie @merlimat thanks for the review PTAL.

@@ -164,6 +175,22 @@ public ManagedCursor getSlowestReader() {
}
}

public boolean hasCursors() {
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 becoming a bit confusing now to understand the meaning of hasCursors() and isEmpty(). We should be use a more unambiguous/explicit naming here.

My suggestion:

  • hasCursors() --> isEmtpy() // Since we're talking about a cursors container
  • isEmpty() --> hasDurableCursors()

(and fix the reversal of the boolean logic)

// Fallback to read lock
stamp = rwLock.readLock();
try {
isEmpty = cursors.isEmpty();
Copy link
Contributor

Choose a reason for hiding this comment

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

Wouldn't the logic be the reverse here? We're checking isEmpty but the method should return true if it's not empty, no?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ya just used the same logic as "isEmpty" but the name of the method hints at a different semantic meaning.

@sijie
Copy link
Member

sijie commented Apr 23, 2020

@jerrypeng @merlimat I think this should be a flag to allow people to choose whether to include readers for backlog quota consideration or not. The behavior of a non-durable cursor reading a managed ledger should be close to the behavior of a file reader reading a file in the filesystem. If a reader is actively reading a file, the file is still accessible even it is deleted. If there is a reader active in the manager ledger, the data shouldn't be deleted if the reader hasn't read it. Otherwise, it is a violation of the correctness of a "distributed log".

Thus, this behavior shouldn't be changed by default. If there is a requirement of excluding readers from backlog consideration, we should use a setting to let people opt-in this feature.

@jerrypeng
Copy link
Contributor Author

jerrypeng commented Apr 23, 2020

@sijie the default behavior of stored data in pulsar is not exactly the same as data stored in a file. By default data is not persisted in Pulsar at all. Only if a topic has a subscription or retention configured is the data actually persisted. Readers are by definition ephemeral and have no persistent state from a brokers perspective, thus if you want to Readers to able to read a certain amount of data, the user should configure the appropriate retention. Readers are responsible for reading the data and not to determine how much data is retained. I believe those are two separate concepts. I would also argue that if you are using the Reader API and you want some level of guarantee of reading the data, you would have to configure retention. If you don't and the Reader dies, the data is not guaranteed to be there when the Reader comes back.

@merlimat
Copy link
Contributor

The behavior of a non-durable cursor reading a managed ledger should be close to the behavior of a file reader reading a file in the filesystem. If a reader is actively reading a file, the file is still accessible even it is deleted. If there is a reader active in the manager ledger, the data shouldn't be deleted if the reader hasn't read it. Otherwise, it is a violation of the correctness of a "distributed log".

There is absolutely no violation of correctness. The contract is clear and it's that a reader can read all the data that is being retained based on max time and size.

Above the data retention thresholds, there is no guarantee to be able to read the data.

That is very important, because a reader should not under any condition to get the backlog quota filled up. In particular a reader should not have impact over a producer. If we allow a reader to stay connected and have the data retained, then when would that be the limit? and what would be the action after the limit?

The current behavior is an accidental one, that in any case it's not something one can rely upon. (eg: a brief disconnection of the reader will cause the data to be deleted anyway).

@sijie
Copy link
Member

sijie commented Apr 23, 2020

The contract is clear and it's that a reader can read all the data that is being retained based on max time and size.

This only covers 50% of the contract. If a reader is scanning a log, it should receive events in order and there are no data gaps.

@jerrypeng @merlimat I am not arguing about data retention. Please get my point correctly. My argument is about the "expected" behavior when a reader attached to a "distributed log". The reader should be able to read the messages from this cursor without missing any data. This is the behavior you can get from a storage system like any local or distributed file system. If we are saying Pulsar is an event/stream storage system, that is the correct behavior we should provide.

If we allow a reader to stay connected and have the data retained, then when would that be the limit? and what would be the action after the limit?

If you look into any file system, when you are opening a file to read and there is a background process delete the file for whatever reason, the reader can still read the file until it is closed. The file system only reclaims disk spaces when the last active open file descriptor is closed.

The problem of this change here is not about data retention. The problem of this change is that it introduces uncertainty in Reader API where people can not trust. The Reader API is effectively a storage API that people rely on building stateful applications. We should take this seriously and follow a common semantic that most of the storage systems provide.

As I said, if you want to relax this contract, it should be done via a flag. We should allow users to decide which behavior they want to choose. If I am building stateful applications that use Pulsar as the source of truth, I don't want to see those uncertainties.

@merlimat
Copy link
Contributor

This only covers 50% of the contract. If a reader is scanning a log, it should receive events in order and there are no data gaps.

That's not what the contract is. And it's not how the reader was designed. Take a look at #355

Introduce a new low-level API that allow applications to read through all
the messages available in a topic without the need of creating a 
subscription.

A reader is a new entity in the Pulsar client API that will only exists 
when connected to a broker. Reader will be able to decide at which
message id to start reading, and it will read all the messages after that.

A reader is only useful in practice if the retention time is set to keep
the data available for a given amount of time. The reader being 
connected will not prevent the broker to delete the data once the 
retention period expires.

That is to say: "The reader should receive events in order and there are no data gaps, while operating within the configured constraints".

Again, I want to be absolutely clear that this change is not relaxing any guarantee over current behavior.

We should not be mixing the life-cycle of 2 different components:

  • The client Reader object is valid from when it's created until one calls close()
  • Whenever a client Reader is connected to a broker, there will be a "NonDurableSubscription" associated. Its lifecycle is tied to the state of the TCP connection, or an explicit close.

Right now, the data is only retained within the scope of the NonDurableSubscription. That is not giving any meaningful guarantee to to the Reader concept, since a TCP re-connection can happen at any time for many different reasons.

Since the fact that "NonDurableSubscription" retaining data goes directly against the stated goal, and it doesn't provide any guarantees, it should be regarded as a bug, not as an optional feature.

If you look into any file system, when you are opening a file to read and there is a background process delete the file for whatever reason, the reader can still read the file until it is closed. The file system only reclaims disk spaces when the last active open file descriptor is closed.

A file system is also operating within system constraints. When the disk (or user quota) is full, some actions will have to be taken.

@sijie
Copy link
Member

sijie commented Apr 27, 2020

A file system is also operating within system constraints. When the disk (or user quota) is full, some actions will have to be taken.

Correct. But the semantic is still maintained. The file is not deleted until the file description is closed.

If a reader is active on a stream, the data after the reader position shouldn't be deleted. I think this is the minimum requirement that a reader should provide. Again, I am fine with introducing this change. But a flag should be added to enable this and the default behavior is that the data shouldn't be removed if there is a reader is active.

@merlimat
Copy link
Contributor

I think this is the minimum requirement that a reader should provide.

But a flag should be added to enable this and the default behavior is that the data shouldn't be removed if there is a reader is active.

As I explained above, this is not how the reader works right now. And there is no way to "guarantee" that behavior right now.

I don't think a flag would change that. Supporting by default a "half" guarantee doesn't give anything that an application could rely upon.

If a reader is active on a stream, the data after the reader position shouldn't be deleted

The definition of active is tricky. Since by definition the reader is tied to the TCP connection.
If one needs a stronger retention semantic, it should be using a "subscription" instead.

To be clear, the behavior of the Pulsar reader was modeled over the Kafka consumer where the same scenario happens: data is deleted irrespectively of whether consumers are still consuming or not.

If we want to change the implementation of the Pulsar topic reader to support a different semantic that today (eg: guaranteed data for the reader), that would be a longer discussion and it would involve a re-design of the reader that would have to be a "durable" resource instead of an ephemeral one.

In my view, at that point we'd be essentially looking at an exclusive subscription with very tiny semantic differences.

@jerrypeng
Copy link
Contributor Author

jerrypeng commented May 8, 2020

@sijie any additional thoughts? @merlimat and I strongly think this is the correct approach for Readers.

@sijie
Copy link
Member

sijie commented May 9, 2020

@jerrypeng I am not convinced about this.

  1. the Kafka model isn't the right model. I don't think we should model our reader model over Kafka's model.
  2. In order to make Pulsar as real stream storage, we need to take serious consideration on the reader's behavior. I agree we might need more enhancements to strengthen the reader guarantee. But we shouldn't simply drop an existing behavior.
  3. For behavior compatibility consideration, we should at least add a flag to control this behavior.

So as I said, we need to provide a flag to control the behavior. Only that we can be able to strengthen the reader semantic in the future. Dropping an existing behavior directly doesn't seem to be the right approach for now.

Step back - what is the real concern preventing you adding a flag?

@merlimat
Copy link
Contributor

Step back - what is the real concern preventing you adding a flag?

Giving the false impression that data is retained, when in fact is not.

Hoping that a reconnection doesn't happen is not a guarantee.

As I already said, the Reader concept was designed in this way. If we want to change or augment the semantics and the guarantees, a flag per-se doesn't accomplish anything.

In order to make Pulsar as real stream storage, we need to take serious consideration on the reader's behavior.

Again, as I said, there are different ways to guarantee the data retention. Time based is one, a subscription is another. What your describing is essentially a reader that behaves like a subscription.. Then why not just use a subscription if you want to make sure to read the data no matter what?

@sijie
Copy link
Member

sijie commented May 13, 2020

What your describing is essentially a reader that behaves like a subscription.. Then why not just use a subscription if you want to make sure to read the data no matter what?

No. What I am describing is not a subscription. When people are using a reader, it is essentially saying: "I don't want pulsar to manage cursors. I will be responsible for managing cursors.". In this context, it is a reader, not a subscription. What I am trying to ensure is if there is a reader actively reading events, we should be able to make sure the continuity of the events it read.

As I already said, the Reader concept was designed in this way.

Step back. I am not going to argue about what should be the correct behavior. Because both of us have our considerations behind it. What I have suggested of adding a flag is a common process of changing existing behavior. The current behavior of a reader is that readers are included in the backlog quota. This is the behavior that we have delivered to the users. If we want to change the behavior, a flag is better to be added to ensure consistent behavior as before. It is a quite common practice to apply for maintaining open source projects.

@codelipenghui codelipenghui modified the milestones: 2.6.0, 2.7.0 Jun 4, 2020
@jerrypeng
Copy link
Contributor Author

@sijie I have added some documentation. Can you please check to see if it is appropriate? If it is can we push this PR through as discussed?

@jerrypeng
Copy link
Contributor Author

/pulsarbot run-failure-checks

1 similar comment
@jerrypeng
Copy link
Contributor Author

/pulsarbot run-failure-checks

@codelipenghui codelipenghui modified the milestones: 2.7.0, 2.6.0 Jun 8, 2020
@codelipenghui codelipenghui merged commit bcfa7f0 into apache:master Jun 8, 2020
cdbartholomew pushed a commit to kafkaesque-io/pulsar that referenced this pull request Jul 24, 2020
### Motivation

When using Readers API, backlog quotas are still enforced on these ephemeral readers.

### Modifications

If a cursor is non-durable then we don't add it to the min-heap we are using keep track of the slowest cursor.  

We use in the min-heap to access the slowest cursor for backlog and quota calculations.  

It is also used to determine up to which ledger we can trim.  Thus, with this change, reader's ephemeral subscriptions won't be preventing ledgers from being clean up as well which is what you expect the behavior to be.

There are some caveats to the above.  Since triggering of trimming of ledgers is piggy packed on top of ledger operations, if there is no new data coming in ledgers will not be trimmed.  The most recently closed ledger we be persisted past any retention.  Perhaps in a subsequent PR, we implement the trimming to be trigger on a schedule as well.
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this pull request Aug 24, 2020
### Motivation

When using Readers API, backlog quotas are still enforced on these ephemeral readers.

### Modifications

If a cursor is non-durable then we don't add it to the min-heap we are using keep track of the slowest cursor.  

We use in the min-heap to access the slowest cursor for backlog and quota calculations.  

It is also used to determine up to which ledger we can trim.  Thus, with this change, reader's ephemeral subscriptions won't be preventing ledgers from being clean up as well which is what you expect the behavior to be.

There are some caveats to the above.  Since triggering of trimming of ledgers is piggy packed on top of ledger operations, if there is no new data coming in ledgers will not be trimmed.  The most recently closed ledger we be persisted past any retention.  Perhaps in a subsequent PR, we implement the trimming to be trigger on a schedule as well.
codelipenghui added a commit to codelipenghui/incubator-pulsar that referenced this pull request Oct 28, 2021
The compactor update the compaction cursor(mark delete) first and then update the `compactionHorizon` of the compacted topic.
During the compaction cursor move forward, the original ledger will be removed if no other durable cursors.
At the same time, if the reader is reading data from the original ledger, the reader will skip the data while the original ledger
been removed, details to see apache#6787. So the reader might skip the compacted data since the
`compactionHorizon` have not updated yet.

The approach is:

1. Update the `compactionHorizon` before the compaction cursor move forward,
   so that the reader will not skip the original data before `compactionHorizon` updated.
   If the broker crashes before the new compacted Ledger ID been persistent,
   after the topic been loaded, the compaction can be trigger again and will not loss any data,
   but we will have an orphan ledger cannot be delete in the BookKeeper cluster.
2. Remove the previous compacted Ledger after the compaction cursor move forward, make sure the new compacted Ledger ID been persistent,
   Otherwise, we might lost compacted ledger if broker crashes.
merlimat pushed a commit that referenced this pull request Nov 2, 2021
#12522)

* Fix the reader skips compacted data which original ledger been removed

The compactor update the compaction cursor(mark delete) first and then update the `compactionHorizon` of the compacted topic.
During the compaction cursor move forward, the original ledger will be removed if no other durable cursors.
At the same time, if the reader is reading data from the original ledger, the reader will skip the data while the original ledger
been removed, details to see #6787. So the reader might skip the compacted data since the
`compactionHorizon` have not updated yet.

The approach is:

1. Update the `compactionHorizon` before the compaction cursor move forward,
   so that the reader will not skip the original data before `compactionHorizon` updated.
   If the broker crashes before the new compacted Ledger ID been persistent,
   after the topic been loaded, the compaction can be trigger again and will not loss any data,
   but we will have an orphan ledger cannot be delete in the BookKeeper cluster.
2. Remove the previous compacted Ledger after the compaction cursor move forward, make sure the new compacted Ledger ID been persistent,
   Otherwise, we might lost compacted ledger if broker crashes.

* Fix checkstyle

* Fix tests.

* Fix test
codelipenghui added a commit to codelipenghui/incubator-pulsar that referenced this pull request Nov 3, 2021
…g ledgers while topic with compaction.

For the non-durable cursor, the ledgers trimming task will cause skip the removed ledgers
to avoid readers introduced backlogs and make sure the data can be removed if over the retention,
more details to see apache#6787.

But for a topic which enabled compaction, this will lead to the reader skips the compacted data.
The new added test can illustrate this problem well. For reading compacted data, reading a message ID
that earlier that the first message ID of the original data is a normal behavior, so we should not
move forward the cursor which will read the compacted data.
hangc0276 pushed a commit that referenced this pull request Nov 4, 2021
#12522)

* Fix the reader skips compacted data which original ledger been removed

The compactor update the compaction cursor(mark delete) first and then update the `compactionHorizon` of the compacted topic.
During the compaction cursor move forward, the original ledger will be removed if no other durable cursors.
At the same time, if the reader is reading data from the original ledger, the reader will skip the data while the original ledger
been removed, details to see #6787. So the reader might skip the compacted data since the
`compactionHorizon` have not updated yet.

The approach is:

1. Update the `compactionHorizon` before the compaction cursor move forward,
   so that the reader will not skip the original data before `compactionHorizon` updated.
   If the broker crashes before the new compacted Ledger ID been persistent,
   after the topic been loaded, the compaction can be trigger again and will not loss any data,
   but we will have an orphan ledger cannot be delete in the BookKeeper cluster.
2. Remove the previous compacted Ledger after the compaction cursor move forward, make sure the new compacted Ledger ID been persistent,
   Otherwise, we might lost compacted ledger if broker crashes.

* Fix checkstyle

* Fix tests.

* Fix test

(cherry picked from commit 74dd9b9)
merlimat pushed a commit that referenced this pull request Nov 4, 2021
…g ledgers while topic with compaction (#12602)

* [Compaction] Do not move the non-durable cursor position when trimming ledgers while topic with compaction.

For the non-durable cursor, the ledgers trimming task will cause skip the removed ledgers
to avoid readers introduced backlogs and make sure the data can be removed if over the retention,
more details to see #6787.

But for a topic which enabled compaction, this will lead to the reader skips the compacted data.
The new added test can illustrate this problem well. For reading compacted data, reading a message ID
that earlier that the first message ID of the original data is a normal behavior, so we should not
move forward the cursor which will read the compacted data.

* Fix checkstyle.

* Fix tests.

* Fix tests.
codelipenghui added a commit that referenced this pull request Nov 5, 2021
…g ledgers while topic with compaction (#12602)

* [Compaction] Do not move the non-durable cursor position when trimming ledgers while topic with compaction.

For the non-durable cursor, the ledgers trimming task will cause skip the removed ledgers
to avoid readers introduced backlogs and make sure the data can be removed if over the retention,
more details to see #6787.

But for a topic which enabled compaction, this will lead to the reader skips the compacted data.
The new added test can illustrate this problem well. For reading compacted data, reading a message ID
that earlier that the first message ID of the original data is a normal behavior, so we should not
move forward the cursor which will read the compacted data.

* Fix checkstyle.

* Fix tests.

* Fix tests.

(cherry picked from commit a6b1b34)
eolivelli pushed a commit to eolivelli/pulsar that referenced this pull request Nov 29, 2021
apache#12522)

* Fix the reader skips compacted data which original ledger been removed

The compactor update the compaction cursor(mark delete) first and then update the `compactionHorizon` of the compacted topic.
During the compaction cursor move forward, the original ledger will be removed if no other durable cursors.
At the same time, if the reader is reading data from the original ledger, the reader will skip the data while the original ledger
been removed, details to see apache#6787. So the reader might skip the compacted data since the
`compactionHorizon` have not updated yet.

The approach is:

1. Update the `compactionHorizon` before the compaction cursor move forward,
   so that the reader will not skip the original data before `compactionHorizon` updated.
   If the broker crashes before the new compacted Ledger ID been persistent,
   after the topic been loaded, the compaction can be trigger again and will not loss any data,
   but we will have an orphan ledger cannot be delete in the BookKeeper cluster.
2. Remove the previous compacted Ledger after the compaction cursor move forward, make sure the new compacted Ledger ID been persistent,
   Otherwise, we might lost compacted ledger if broker crashes.

* Fix checkstyle

* Fix tests.

* Fix test
eolivelli pushed a commit to eolivelli/pulsar that referenced this pull request Nov 29, 2021
…g ledgers while topic with compaction (apache#12602)

* [Compaction] Do not move the non-durable cursor position when trimming ledgers while topic with compaction.

For the non-durable cursor, the ledgers trimming task will cause skip the removed ledgers
to avoid readers introduced backlogs and make sure the data can be removed if over the retention,
more details to see apache#6787.

But for a topic which enabled compaction, this will lead to the reader skips the compacted data.
The new added test can illustrate this problem well. For reading compacted data, reading a message ID
that earlier that the first message ID of the original data is a normal behavior, so we should not
move forward the cursor which will read the compacted data.

* Fix checkstyle.

* Fix tests.

* Fix tests.
eolivelli pushed a commit that referenced this pull request Dec 13, 2021
#12522)

* Fix the reader skips compacted data which original ledger been removed

The compactor update the compaction cursor(mark delete) first and then update the `compactionHorizon` of the compacted topic.
During the compaction cursor move forward, the original ledger will be removed if no other durable cursors.
At the same time, if the reader is reading data from the original ledger, the reader will skip the data while the original ledger
been removed, details to see #6787. So the reader might skip the compacted data since the
`compactionHorizon` have not updated yet.

The approach is:

1. Update the `compactionHorizon` before the compaction cursor move forward,
   so that the reader will not skip the original data before `compactionHorizon` updated.
   If the broker crashes before the new compacted Ledger ID been persistent,
   after the topic been loaded, the compaction can be trigger again and will not loss any data,
   but we will have an orphan ledger cannot be delete in the BookKeeper cluster.
2. Remove the previous compacted Ledger after the compaction cursor move forward, make sure the new compacted Ledger ID been persistent,
   Otherwise, we might lost compacted ledger if broker crashes.

* Fix checkstyle

* Fix tests.

* Fix test

(cherry picked from commit 74dd9b9)
codelipenghui added a commit that referenced this pull request Dec 20, 2021
…g ledgers while topic with compaction (#12602)

* [Compaction] Do not move the non-durable cursor position when trimming ledgers while topic with compaction.

For the non-durable cursor, the ledgers trimming task will cause skip the removed ledgers
to avoid readers introduced backlogs and make sure the data can be removed if over the retention,
more details to see #6787.

But for a topic which enabled compaction, this will lead to the reader skips the compacted data.
The new added test can illustrate this problem well. For reading compacted data, reading a message ID
that earlier that the first message ID of the original data is a normal behavior, so we should not
move forward the cursor which will read the compacted data.

* Fix checkstyle.

* Fix tests.

* Fix tests.

(cherry picked from commit a6b1b34)
eolivelli pushed a commit to eolivelli/pulsar that referenced this pull request Feb 25, 2022
apache#12522)

* Fix the reader skips compacted data which original ledger been removed

The compactor update the compaction cursor(mark delete) first and then update the `compactionHorizon` of the compacted topic.
During the compaction cursor move forward, the original ledger will be removed if no other durable cursors.
At the same time, if the reader is reading data from the original ledger, the reader will skip the data while the original ledger
been removed, details to see apache#6787. So the reader might skip the compacted data since the
`compactionHorizon` have not updated yet.

The approach is:

1. Update the `compactionHorizon` before the compaction cursor move forward,
   so that the reader will not skip the original data before `compactionHorizon` updated.
   If the broker crashes before the new compacted Ledger ID been persistent,
   after the topic been loaded, the compaction can be trigger again and will not loss any data,
   but we will have an orphan ledger cannot be delete in the BookKeeper cluster.
2. Remove the previous compacted Ledger after the compaction cursor move forward, make sure the new compacted Ledger ID been persistent,
   Otherwise, we might lost compacted ledger if broker crashes.

* Fix checkstyle

* Fix tests.

* Fix test

(cherry picked from commit 74dd9b9)
(cherry picked from commit df68493)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants