This is an automated email from the ASF dual-hosted git repository. jianghaiting pushed a commit to branch branch-2.10 in repository https://gitbox.apache.org/repos/asf/pulsar.git
commit 917f997bef9fe78b722ec85b7cf7ae8643295121 Author: Yong Zhang <[email protected]> AuthorDate: Wed Sep 14 10:41:13 2022 +0800 [fix][tiered-storage] Don't cleanup data when offload met Metastore exception (#17512) * [fix][tiered-storage] Don't cleanup data when offload met BadVersion --- *Motivation* There have two ways that will cause the offload data cleanup. One is met offload conflict exception, and another is completeLedgerInfoForOffloaded reaches max retry time and throws zookeeper exceptions. We retry the zookeeper operation on connection loss exception. We should be careful about this exception, because we may loss data if the metadata update successfully. When a MetaStore exception happens, we can not make sure the metadata update is failed or not. Because we have a retry on the connection loss, it is possible to get a BadVersion or other exception after retrying. So we don't clean up the data if this happens. *Modification* - don't delete data if has meta store exception * log error when skip deleting * improve logs (cherry picked from commit c2588ba6a07c05b2c1ce9cc8a4cf33e5b4a2755d) --- .../apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 438539bfdd5..73b204f4749 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -2996,8 +2996,21 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { scheduledExecutor, name) .whenComplete((ignore2, exception) -> { if (exception != null) { - log.error("[{}] Failed to offload data for the ledgerId {}", + Throwable e = FutureUtil.unwrapCompletionException(exception); + if (e instanceof MetaStoreException) { + // When a MetaStore exception happens, we can not make sure the metadata + // update is failed or not. Because we have a retry on the connection loss, + // it is possible to get a BadVersion or other exception after retrying. + // So we don't clean up the data if it has metadata operation exception. + log.error("[{}] Failed to update offloaded metadata for the ledgerId {}, " + + "the offloaded data will not be cleaned up", name, ledgerId, exception); + return; + } else { + log.error("[{}] Failed to offload data for the ledgerId {}, " + + "clean up the offloaded data", + name, ledgerId, exception); + } cleanupOffloaded( ledgerId, uuid, driverName, driverMetadata,
