Skip to content

Conversation

@lordcheng10
Copy link
Contributor

@lordcheng10 lordcheng10 commented Jul 6, 2022

Motivation

In the log we found No such ledger exception:
image

The reasons are as follows:
1. Frequent Full GC occurs on the broker, causing zk to time out:
22:29:06.119 [main-EventThread] ERROR org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase - ZooKeeper client connection to the ZooKeeper server has expired!
2.When the create leadger is completed, the ledger will be put into ledgers and update currentLedger,the updateLedgersListAfterRollover method is executed, and the metadata to zookeeper fails to write:
image

log.info("[{}] Created new ledger {}", name, lh.getId());
ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
currentLedger = lh;
currentLedgerEntries = 0;
currentLedgerSize = 0;

3.In the failure callback method operationFailed, the corresponding ledger will be removed from the ledgers, but the currentLedger still points to the ledger that failed to create, but the ledger will be deleted here through bookKeeper.asyncDeleteLedger. When reading data, it will be read through the currentLedger. Since the ledger has been deleted, the final error is reported: No such ledger

public void operationFailed(MetaStoreException e) {
log.warn("[{}] Error updating meta data with the new list of ledgers: {}", name, e.getMessage());
// Remove the ledger, since we failed to update the list
ledgers.remove(lh.getId());
mbean.startDataLedgerDeleteOp();
bookKeeper.asyncDeleteLedger(lh.getId(), (rc1, ctx1) -> {
mbean.endDataLedgerDeleteOp();
if (rc1 != BKException.Code.OK) {
log.warn("[{}] Failed to delete ledger {}: {}", name, lh.getId(),
BKException.getMessage(rc1));
}
}, null);
if (e instanceof BadVersionException) {

Modifications

After the create ledger is created and the metadata is written to the zookeeper successfully, the currentLedger is updated.

Documentation

Check the box below or label this PR directly.

Need to update docs?

  • doc-required
    (Your PR needs to update docs and you will update later)

  • doc-not-needed
    (Please explain why)

  • doc
    (Your PR contains doc changes)

  • doc-complete
    (Docs have been already added)

@github-actions
Copy link

github-actions bot commented Jul 6, 2022

@lordcheng10 Please provide a correct documentation label for your PR.
Instructions see Pulsar Documentation Label Guide.

@github-actions github-actions bot added doc-label-missing doc-not-needed Your PR changes do not impact docs and removed doc-label-missing labels Jul 6, 2022
@lordcheng10
Copy link
Contributor Author

@hangc0276 PTAL,thanks!

@lordcheng10
Copy link
Contributor Author

@codelipenghui @eolivelli PTAL,thanks!

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!

@lordcheng10
Copy link
Contributor Author

/pulsarbot run-failure-checks

2 similar comments
@lordcheng10
Copy link
Contributor Author

/pulsarbot run-failure-checks

@lordcheng10
Copy link
Contributor Author

/pulsarbot run-failure-checks

log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
}
ledgersStat = stat;
ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
Copy link
Contributor

Choose a reason for hiding this comment

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

@lordcheng10 Adding to the ledgers map cannot be done here, because that map is used when writing to the z-node (eg: buildManagedLedgerInfo(Map<Long, LedgerInfo> ledgers).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed. PTAL,thanks! @merlimat

Copy link
Member

Choose a reason for hiding this comment

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

I still have concerns because in this method, if the ledger is not persisted to the metadata, it means the ledger cannot be used. But we put it in the ledger map; it is not persistent and may be used by other operations under multi-threaded conditions.

However, I'm not sure what it will affect. Just a minor concern. Please feel free to go on.

Copy link
Contributor Author

@lordcheng10 lordcheng10 Jul 7, 2022

Choose a reason for hiding this comment

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

Your concern is right, I will try to update the zookeeper metadata with a temporary ledgersTmp, and update the ledgers after the zookeeper is successfully written. @mattisonchao @merlimat

Copy link
Contributor

Choose a reason for hiding this comment

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

Agree with @mattisonchao. This may lead to data loss.

Putting the newly created ledger into the NavigableMap<Long, LedgerInfo> ledgers, the new ledger can be seen immediately which means can write data into this ledger, and then if the meta-store operation failed, the ledger will be removed from both the ledgers and bookie, all data in this ledger are lost.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@mattisonchao
Copy link
Member

@hangc0276

This pull request made a lot of change after your review. Could you have time to review it again?

@mattisonchao mattisonchao requested a review from hangc0276 July 7, 2022 08:34
@lordcheng10
Copy link
Contributor Author

/pulsarbot run-failure-checks

@lordcheng10
Copy link
Contributor Author

lordcheng10 commented Jul 7, 2022

I think we are not handling this condition well:
If the process runs at this line, we continue to retry later. But at the same time, the ledgers is changed. However, we still use old data to update. I'm not sure if it is a big problem. But I think we have to clarify it.
I think I have to give a change request. @lordcheng10 Could you help to confirm it?

This is a problem, I made a modification, please review it again.
The metadata information written to zookeeper should be constructed after the metadataMutex is obtained. @mattisonchao

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.

LGTM +1

@lordcheng10
Copy link
Contributor Author

/pulsarbot run-failure-checks

4 similar comments
@lordcheng10
Copy link
Contributor Author

/pulsarbot run-failure-checks

@lordcheng10
Copy link
Contributor Author

/pulsarbot run-failure-checks

@lordcheng10
Copy link
Contributor Author

/pulsarbot run-failure-checks

@lordcheng10
Copy link
Contributor Author

/pulsarbot run-failure-checks

@Jason918 Jason918 merged commit 2e5fbbc into apache:master Jul 8, 2022
codelipenghui pushed a commit that referenced this pull request Jul 10, 2022
* fix No such ledger exception

* move currentLedgerEntries and currentLedgerSize

* move ledgers.put  to operationComplete

* fix ledgers.put

* use ledgersTmp to write zookkeeper,after sucess,update ldgers

* update updateLedgersListAfterRollover

* put lh to ledgersTmp

* extend buildManagedLedgerInfo

* getManagedLedgerInfo(newLedger) after get metadataMutex

(cherry picked from commit 2e5fbbc)
zymap pushed a commit to zymap/pulsar that referenced this pull request Jul 11, 2022
* fix No such ledger exception

* move currentLedgerEntries and currentLedgerSize

* move ledgers.put  to operationComplete

* fix ledgers.put

* use ledgersTmp to write zookkeeper,after sucess,update ldgers

* update updateLedgersListAfterRollover

* put lh to ledgersTmp

* extend buildManagedLedgerInfo

* getManagedLedgerInfo(newLedger) after get metadataMutex
nicoloboschi pushed a commit to datastax/pulsar that referenced this pull request Jul 11, 2022
* fix No such ledger exception

* move currentLedgerEntries and currentLedgerSize

* move ledgers.put  to operationComplete

* fix ledgers.put

* use ledgersTmp to write zookkeeper,after sucess,update ldgers

* update updateLedgersListAfterRollover

* put lh to ledgersTmp

* extend buildManagedLedgerInfo

* getManagedLedgerInfo(newLedger) after get metadataMutex

(cherry picked from commit 2e5fbbc)
(cherry picked from commit 41c0cf3)
wuxuanqicn pushed a commit to wuxuanqicn/pulsar that referenced this pull request Jul 14, 2022
* fix No such ledger exception

* move currentLedgerEntries and currentLedgerSize

* move ledgers.put  to operationComplete

* fix ledgers.put

* use ledgersTmp to write zookkeeper,after sucess,update ldgers

* update updateLedgersListAfterRollover

* put lh to ledgersTmp

* extend buildManagedLedgerInfo

* getManagedLedgerInfo(newLedger) after get metadataMutex
@mattisonchao mattisonchao added the cherry-picked/branch-2.9 Archived: 2.9 is end of life label Jul 15, 2022
mattisonchao pushed a commit that referenced this pull request Jul 15, 2022
* fix No such ledger exception

* move currentLedgerEntries and currentLedgerSize

* move ledgers.put  to operationComplete

* fix ledgers.put

* use ledgersTmp to write zookkeeper,after sucess,update ldgers

* update updateLedgersListAfterRollover

* put lh to ledgersTmp

* extend buildManagedLedgerInfo

* getManagedLedgerInfo(newLedger) after get metadataMutex

(cherry picked from commit 2e5fbbc)
BewareMyPower pushed a commit that referenced this pull request Jul 29, 2022
* fix No such ledger exception

* move currentLedgerEntries and currentLedgerSize

* move ledgers.put  to operationComplete

* fix ledgers.put

* use ledgersTmp to write zookkeeper,after sucess,update ldgers

* update updateLedgersListAfterRollover

* put lh to ledgersTmp

* extend buildManagedLedgerInfo

* getManagedLedgerInfo(newLedger) after get metadataMutex

(cherry picked from commit 2e5fbbc)
@BewareMyPower BewareMyPower added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Jul 29, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

9 participants