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

Fix lost message issue due to ledger rollover. #14664

Merged
merged 4 commits into from
Mar 12, 2022

Conversation

Technoboy-
Copy link
Contributor

@Technoboy- Technoboy- commented Mar 11, 2022

Motivation

pre-require : User config managedLedgerMaxLedgerRolloverTimeMinutes > 0.

Then, if ManagedLedger creates a ledger in the below case :

} else if (state == State.ClosedLedger) {
// No ledger and no pending operations. Create a new ledger
log.info("[{}] Creating a new ledger", name);
if (STATE_UPDATER.compareAndSet(this, State.ClosedLedger, State.CreatingLedger)) {
this.lastLedgerCreationInitiationTimestamp = System.currentTimeMillis();
mbean.startDataLedgerCreateOp();
asyncCreateLedger(bookKeeper, config, digestType, this, Collections.emptyMap());
}

The ManagedLedger state now is CreatingLedger . At this moment, rollover is triggered, it will set the state to ClosingLedger (line-1674)

public void rollCurrentLedgerIfFull() {
log.info("[{}] Start checking if current ledger is full", name);
if (currentLedgerEntries > 0 && currentLedgerIsFull()) {
STATE_UPDATER.set(this, State.ClosingLedger);
currentLedger.asyncClose(new AsyncCallback.CloseCallback() {
@Override
public void closeComplete(int rc, LedgerHandle lh, Object o) {
checkArgument(currentLedger.getId() == lh.getId(), "ledgerId %s doesn't match with "
+ "acked ledgerId %s", currentLedger.getId(), lh.getId());
if (rc == BKException.Code.OK) {
log.debug("Successfully closed ledger {}", lh.getId());
} else {
log.warn("Error when closing ledger {}. Status={}", lh.getId(), BKException.getMessage(rc));
}
ledgerClosed(lh);
createLedgerAfterClosed();
}
}, System.nanoTime());
}
}

And invoke createLedgerAfterClosed -> isNeededCreateNewLedgerAfterCloseLedger is passing,
because isNeededCreateNewLedgerAfterCloseLedger checks CreatingLedger and LedgerOpened state, but the current state is ClosingLedger:

synchronized void createLedgerAfterClosed() {
if (isNeededCreateNewLedgerAfterCloseLedger()) {
log.info("[{}] Creating a new ledger", name);
STATE_UPDATER.set(this, State.CreatingLedger);
this.lastLedgerCreationInitiationTimestamp = System.currentTimeMillis();
mbean.startDataLedgerCreateOp();
// Use the executor here is to avoid use the Zookeeper thread to create the ledger which will lead
// to deadlock at the zookeeper client, details to see https://github.com/apache/pulsar/issues/13736
this.executor.execute(() ->
asyncCreateLedger(bookKeeper, config, digestType, this, Collections.emptyMap()));
}
}
boolean isNeededCreateNewLedgerAfterCloseLedger() {
final State state = STATE_UPDATER.get(this);
if (state != State.CreatingLedger && state != State.LedgerOpened) {
return true;
}
return false;
}

So it will also create a new ledger. And result in the message being lost. It's the same result with #12221

Detail log:

2022-03-04T01:41:49.908333235+02:00 2022-03-03T23:41:49,907 [BookKeeperClientWorker-OrderedExecutor-3-0] INFO  org.apache.bookkeeper.mledger.impl.OpAddEntry - [public/default/persistent/task_topic-partition-0] Closing ledger 1846355 for being full
2022-03-04T01:41:49.921447018+02:00 2022-03-03T23:41:49,920 [BookKeeperClientWorker-OrderedExecutor-3-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/default/persistent/task_topic-partition-0] Creating a new ledger
2022-03-04T01:41:49.921482783+02:00 2022-03-03T23:41:49,921 [BookKeeperClientWorker-OrderedExecutor-3-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/itomdipostload/persistent/task_topic-partition-0] Creating ledger, metadata: {component=[109, 97, 110, 97, 103, 101, 100, 45, 108, 101, 100, 103, 101, 114], pulsar/managed-ledger=[112, 117, 98, 108, 105, 99, 47, 105, 116, 111, 109, 100, 105, 112, 111, 115, 116, 108, 111, 97, 100, 47, 112, 101, 114, 115, 105, 115, 116, 101, 110, 116, 47, 100, 105, 95, 112, 111, 115, 116, 108, 111, 97, 100, 95, 116, 97, 115, 107, 95, 116, 111, 112, 105, 99, 45, 112, 97, 114, 116, 105, 116, 105, 111, 110, 45, 48], application=[112, 117, 108, 115, 97, 114]} - metadata ops timeout : 60 seconds
2022-03-04T01:41:49.926896931+02:00 2022-03-03T23:41:49,926 [bookkeeper-ml-scheduler-OrderedScheduler-2-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/default/persistent/task_topic-partition-0] Start checking if current ledger is full
2022-03-04T01:41:49.926944096+02:00 2022-03-03T23:41:49,926 [BookKeeperClientWorker-OrderedExecutor-3-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/default/persistent/task_topic-partition-0] Creating a new ledger
2022-03-04T01:41:49.927043799+02:00 2022-03-03T23:41:49,926 [BookKeeperClientWorker-OrderedExecutor-3-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/default/persistent/task_topic-partition-0] Creating ledger, metadata: {component=[109, 97, 110, 97, 103, 101, 100, 45, 108, 101, 100, 103, 101, 114], pulsar/managed-ledger=[112, 117, 98, 108, 105, 99, 47, 105, 116, 111, 109, 100, 105, 112, 111, 115, 116, 108, 111, 97, 100, 47, 112, 101, 114, 115, 105, 115, 116, 101, 110, 116, 47, 100, 105, 95, 112, 111, 115, 116, 108, 111, 97, 100, 95, 116, 97, 115, 107, 95, 116, 111, 112, 105, 99, 45, 112, 97, 114, 116, 105, 116, 105, 111, 110, 45, 48], application=[112, 117, 108, 115, 97, 114]} - metadata ops timeout : 60 seconds
2022-03-04T01:41:49.936780564+02:00 2022-03-03T23:41:49,936 [main-EventThread] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/default/persistent/task_topic-partition-0] Created new ledger 1846376
2022-03-04T01:41:49.944880616+02:00 2022-03-03T23:41:49,944 [main-EventThread] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/default/persistent/task_topic-partition-0] Created new ledger 1846377

Modification

  • Rollover only with ledger LedgerOpened state.
  • Modify log to help troubleshoot problem.

Documentation

  • no-need-doc

@Technoboy- Technoboy- self-assigned this Mar 11, 2022
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Mar 11, 2022
@codelipenghui codelipenghui added this to the 2.10.0 milestone Mar 11, 2022
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

good catch and great work

Copy link
Contributor

@hangc0276 hangc0276 left a comment

Choose a reason for hiding this comment

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

Nice Catch.

Copy link
Member

@mattisonchao mattisonchao left a comment

Choose a reason for hiding this comment

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

Great Catch!

BewareMyPower added a commit to streamnative/kop that referenced this pull request Mar 14, 2022
…dger (#1176)

Fixes #1175

### Motivation

The `ManagedLedger#rollCurrentLedgerIfFull` can only work when the managed ledger's state is `LedgerOpened` since apache/pulsar#14664. The corner case described in #894 is still possible when some entries are sent but not persisted after a rollover and retention works. Therefore, we should still keep the `testListOffsetForEmptyRolloverLedger` to avoid the regression in KoP.

### Modifications

- Bump Pulsar to 2.10.0.0-rc12, which includes apache/pulsar#14664
- Change the managed ledger's state to `LedgerOpened` in `testListOffsetForEmptyRolloverLedger`.
BewareMyPower added a commit to streamnative/kop that referenced this pull request Mar 14, 2022
…dger (#1176)

Fixes #1175

### Motivation

The `ManagedLedger#rollCurrentLedgerIfFull` can only work when the managed ledger's state is `LedgerOpened` since apache/pulsar#14664. The corner case described in #894 is still possible when some entries are sent but not persisted after a rollover and retention works. Therefore, we should still keep the `testListOffsetForEmptyRolloverLedger` to avoid the regression in KoP.

### Modifications

- Bump Pulsar to 2.10.0.0-rc12, which includes apache/pulsar#14664
- Change the managed ledger's state to `LedgerOpened` in `testListOffsetForEmptyRolloverLedger`.

(cherry picked from commit bb37893)
@gaoran10 gaoran10 added the cherry-picked/branch-2.9 Archived: 2.9 is end of life label Mar 14, 2022
gaoran10 pushed a commit that referenced this pull request Mar 14, 2022
codelipenghui pushed a commit that referenced this pull request Mar 14, 2022
Demogorgon314 pushed a commit to Demogorgon314/kop that referenced this pull request Mar 15, 2022
…dger (streamnative#1176)

Fixes streamnative#1175

### Motivation

The `ManagedLedger#rollCurrentLedgerIfFull` can only work when the managed ledger's state is `LedgerOpened` since apache/pulsar#14664. The corner case described in streamnative#894 is still possible when some entries are sent but not persisted after a rollover and retention works. Therefore, we should still keep the `testListOffsetForEmptyRolloverLedger` to avoid the regression in KoP.

### Modifications

- Bump Pulsar to 2.10.0.0-rc12, which includes apache/pulsar#14664
- Change the managed ledger's state to `LedgerOpened` in `testListOffsetForEmptyRolloverLedger`.

(cherry picked from commit bb37893)
Demogorgon314 added a commit to streamnative/kop that referenced this pull request Mar 15, 2022
…dger (#1179)

Fixes #1175

### Motivation

The `ManagedLedger#rollCurrentLedgerIfFull` can only work when the managed ledger's state is `LedgerOpened` since apache/pulsar#14664. The corner case described in #894 is still possible when some entries are sent but not persisted after a rollover and retention works. Therefore, we should still keep the `testListOffsetForEmptyRolloverLedger` to avoid the regression in KoP.

### Modifications

- Bump Pulsar to 2.10.0.0-rc12, which includes apache/pulsar#14664
- Change the managed ledger's state to `LedgerOpened` in `testListOffsetForEmptyRolloverLedger`.

(cherry picked from commit bb37893)

* Bump pulsar to 2.9.2.8

Co-authored-by: Yunze Xu <[email protected]>
codelipenghui pushed a commit that referenced this pull request Mar 16, 2022
nicoloboschi pushed a commit to datastax/pulsar that referenced this pull request Mar 16, 2022
(cherry picked from commit ad2cc2d)
(cherry picked from commit 58e13af)
eolivelli pushed a commit to eolivelli/kop that referenced this pull request Mar 21, 2022
…dger (streamnative#1176)

Fixes streamnative#1175

### Motivation

The `ManagedLedger#rollCurrentLedgerIfFull` can only work when the managed ledger's state is `LedgerOpened` since apache/pulsar#14664. The corner case described in streamnative#894 is still possible when some entries are sent but not persisted after a rollover and retention works. Therefore, we should still keep the `testListOffsetForEmptyRolloverLedger` to avoid the regression in KoP.

### Modifications

- Bump Pulsar to 2.10.0.0-rc12, which includes apache/pulsar#14664
- Change the managed ledger's state to `LedgerOpened` in `testListOffsetForEmptyRolloverLedger`.
eolivelli pushed a commit to eolivelli/kop that referenced this pull request Mar 21, 2022
…dger (streamnative#1176)

Fixes streamnative#1175

### Motivation

The `ManagedLedger#rollCurrentLedgerIfFull` can only work when the managed ledger's state is `LedgerOpened` since apache/pulsar#14664. The corner case described in streamnative#894 is still possible when some entries are sent but not persisted after a rollover and retention works. Therefore, we should still keep the `testListOffsetForEmptyRolloverLedger` to avoid the regression in KoP.

### Modifications

- Bump Pulsar to 2.10.0.0-rc12, which includes apache/pulsar#14664
- Change the managed ledger's state to `LedgerOpened` in `testListOffsetForEmptyRolloverLedger`.
Nicklee007 pushed a commit to Nicklee007/pulsar that referenced this pull request Apr 20, 2022
eolivelli pushed a commit to datastax/starlight-for-kafka that referenced this pull request May 9, 2022
…dger (#1176)

Fixes #1175

The `ManagedLedger#rollCurrentLedgerIfFull` can only work when the managed ledger's state is `LedgerOpened` since apache/pulsar#14664. The corner case described in streamnative/kop#894 is still possible when some entries are sent but not persisted after a rollover and retention works. Therefore, we should still keep the `testListOffsetForEmptyRolloverLedger` to avoid the regression in KoP.

- Bump Pulsar to 2.10.0.0-rc12, which includes apache/pulsar#14664
- Change the managed ledger's state to `LedgerOpened` in `testListOffsetForEmptyRolloverLedger`.

(cherry picked from commit bb37893)
eolivelli pushed a commit to datastax/starlight-for-kafka that referenced this pull request May 9, 2022
…dger (#1176)

Fixes #1175

The `ManagedLedger#rollCurrentLedgerIfFull` can only work when the managed ledger's state is `LedgerOpened` since apache/pulsar#14664. The corner case described in streamnative/kop#894 is still possible when some entries are sent but not persisted after a rollover and retention works. Therefore, we should still keep the `testListOffsetForEmptyRolloverLedger` to avoid the regression in KoP.

- Bump Pulsar to 2.10.0.0-rc12, which includes apache/pulsar#14664
- Change the managed ledger's state to `LedgerOpened` in `testListOffsetForEmptyRolloverLedger`.

(cherry picked from commit bb37893)
(cherry picked from commit 47e332b)
@Jason918 Jason918 added the cherry-picked/branch-2.7 Archived: 2.7 is end of life label Jul 27, 2022
aloyszhang pushed a commit to aloyszhang/pulsar that referenced this pull request Aug 5, 2022
Squash merge branch 'lost-data' into '2.8.1'
修复Broker 丢数据问题(apache#14664)

TAPD: --bug=097504477
@Technoboy- Technoboy- deleted the fix-ledger-rollover-issue branch August 10, 2022 05:52
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker cherry-picked/branch-2.7 Archived: 2.7 is end of life cherry-picked/branch-2.8 Archived: 2.8 is end of life cherry-picked/branch-2.9 Archived: 2.9 is end of life doc-not-needed Your PR changes do not impact docs release/2.7.5 release/2.8.4 release/2.9.2
Projects
None yet
Development

Successfully merging this pull request may close these issues.