From 0c1a8916fe7e1c2824c458a3d487fee0f3beaf1d Mon Sep 17 00:00:00 2001 From: horizonzy Date: Sat, 28 May 2022 14:10:20 +0800 Subject: [PATCH 01/43] init snip-145 --- conf/broker.conf | 18 +- .../mledger/ManagedLedgerConfig.java | 8 + .../bookkeeper/mledger/ManagedTrash.java | 61 ++ .../mledger/ManagedTrashMXBean.java | 44 ++ .../impl/ManagedLedgerFactoryImpl.java | 4 +- .../mledger/impl/ManagedLedgerImpl.java | 339 +++++----- .../mledger/impl/ManagedTrashImpl.java | 619 ++++++++++++++++++ .../mledger/impl/ManagedTrashMXBeanImpl.java | 61 ++ .../impl/ReadOnlyManagedLedgerImpl.java | 7 +- .../src/main/proto/MLDataFormats.proto | 10 + .../metadata/impl/EtcdMetadataStore.java | 2 +- .../pulsar/metadata/impl/ZKMetadataStore.java | 2 +- .../AbstractBatchedMetadataStore.java | 2 +- 13 files changed, 1009 insertions(+), 168 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java diff --git a/conf/broker.conf b/conf/broker.conf index ee1eb3a7a6323..1f9b3389a238b 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -109,7 +109,7 @@ enableBusyWait=false isRunningStandalone= # Name of the cluster to which this broker belongs to -clusterName= +clusterName=horizon-cluster # The maximum number of tenants that each pulsar cluster can create # This configuration is not precise control, in a concurrent scenario, the threshold will be exceeded @@ -119,7 +119,7 @@ maxTenants=0 failureDomainsEnabled=false # Metadata store session timeout in milliseconds -metadataStoreSessionTimeoutMillis=30000 +metadataStoreSessionTimeoutMillis=600000 # Metadata store operation timeout in seconds metadataStoreOperationTimeoutSeconds=30 @@ -232,7 +232,7 @@ subscriptionKeySharedConsistentHashingReplicaPoints=100 # Set the default behavior for message deduplication in the broker # This can be overridden per-namespace. If enabled, broker will reject # messages that were already stored in the topic -brokerDeduplicationEnabled=false +brokerDeduplicationEnabled=true # Maximum number of producer information that it's going to be # persisted for deduplication purposes @@ -515,7 +515,7 @@ delayedDeliveryEnabled=true delayedDeliveryTickTimeMillis=1000 # Whether to enable acknowledge of batch local index. -acknowledgmentAtBatchIndexLevelEnabled=false +acknowledgmentAtBatchIndexLevelEnabled=true # Enable tracking of replicated subscriptions state across clusters. enableReplicatedSubscriptions=true @@ -810,7 +810,7 @@ maxConcurrentHttpRequests=1024 # For example: zk+hierarchical://localhost:2181/ledgers # The metadata service uri list can also be semicolon separated values like below: # zk+hierarchical://zk1:2181;zk2:2181;zk3:2181/ledgers -bookkeeperMetadataServiceUri= +bookkeeperMetadataServiceUri=zk+hierarchical://127.0.0.1:2181/ledgers # Authentication plugin to use when connecting to bookies bookkeeperClientAuthenticationPlugin= @@ -941,10 +941,10 @@ bookkeeperExplicitLacIntervalInMills=0 managedLedgerDefaultEnsembleSize=2 # Number of copies to store for each message -managedLedgerDefaultWriteQuorum=2 +managedLedgerDefaultWriteQuorum=1 # Number of guaranteed copies (acks to wait before write is complete) -managedLedgerDefaultAckQuorum=2 +managedLedgerDefaultAckQuorum=1 # with OpportunisticStriping=true the ensembleSize is adapted automatically to writeQuorum # in case of lack of enough bookies @@ -1440,12 +1440,12 @@ strictBookieAffinityEnabled=false # Zookeeper quorum connection string # Deprecated: use metadataStoreUrl instead -zookeeperServers= +zookeeperServers=127.0.0.1:2181 # Configuration Store connection string # Deprecated. Use configurationMetadataStoreUrl globalZookeeperServers= -configurationStoreServers= +configurationStoreServers=127.0.0.1:2181 # Zookeeper session timeout in milliseconds # Deprecated: use metadataStoreSessionTimeoutMillis diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index e628a253563a1..83723ae9fcf24 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -81,6 +81,7 @@ public class ManagedLedgerConfig { @Getter @Setter private boolean cacheEvictionByMarkDeletedPosition = false; + private int trashDataLimitSize = 1000; public boolean isCreateIfMissing() { return createIfMissing; @@ -683,4 +684,11 @@ public void setInactiveLedgerRollOverTime(int inactiveLedgerRollOverTimeMs, Time this.inactiveLedgerRollOverTimeMs = (int) unit.toMillis(inactiveLedgerRollOverTimeMs); } + public int getTrashDataLimitSize() { + return trashDataLimitSize; + } + + public void setTrashDataLimitSize(int trashDataLimitSize) { + this.trashDataLimitSize = trashDataLimitSize; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java new file mode 100644 index 0000000000000..64e38f93e95d1 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; + +public interface ManagedTrash { + + String MANAGED_LEDGER = "managed-ledger"; + + String MANAGED_CURSOR = "managed-cursor"; + + String SCHEMA = "schema"; + + String DELETABLE_LEDGER_SUFFIX = "DL"; + + String DELETABLE_OFFLOADED_LEDGER_SUFFIX = "DOL"; + + interface TrashMetaStoreCallback { + + void operationComplete(T result); + + void operationFailed(ManagedLedgerException.MetaStoreException e); + } + + String name(); + + CompletableFuture initialize(); + + CompletableFuture appendLedgerTrashData(final long ledgerId, final LedgerInfo context, final String type); + + void asyncUpdateTrashData(Optional> callback); + + void triggerDelete(); + + List getAllArchiveIndex(); + + Map getArchiveData(final long index); + + void asyncClose(final AsyncCallbacks.CloseCallback callback, final Object ctx); +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java new file mode 100644 index 0000000000000..34ac52b99f2bd --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger; + +import org.apache.bookkeeper.common.annotation.InterfaceAudience; +import org.apache.bookkeeper.common.annotation.InterfaceStability; + +/** + * JMX Bean interface for ManagedTrash stats. + */ +@InterfaceAudience.LimitedPrivate +@InterfaceStability.Stable +public interface ManagedTrashMXBean { + + String getName(); + + long getCurrentNumberOfLedgersWaitingToDeleted(); + + long getTotalNumberOfLedgersWaitingToDeleted(); + + void + + long getCurrentNumberOfLedgersWaitingToArchive(); + + long getTotalNumberOfLedgersWaitingToArchive(); + + +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index b6b1bcf3e8d9a..bacee90d9dba1 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -383,7 +383,7 @@ public void asyncOpen(final String name, final ManagedLedgerConfig config, final bookkeeperFactory.get( new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), config.getBookKeeperEnsemblePlacementPolicyProperties())), - store, config, scheduledExecutor, name, mlOwnershipChecker); + store, metadataStore, config, scheduledExecutor, name, mlOwnershipChecker); PendingInitializeManagedLedger pendingLedger = new PendingInitializeManagedLedger(newledger); pendingInitializeLedgers.put(name, pendingLedger); newledger.initialize(new ManagedLedgerInitializeLedgerCallback() { @@ -476,7 +476,7 @@ public void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosi bookkeeperFactory .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), config.getBookKeeperEnsemblePlacementPolicyProperties())), - store, config, scheduledExecutor, managedLedgerName); + store, metadataStore, config, scheduledExecutor, managedLedgerName); roManagedLedger.initializeAndCreateCursor((PositionImpl) startPosition) .thenAccept(roCursor -> callback.openReadOnlyCursorComplete(roCursor, ctx)) 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 277e4df0244b8..67ff37a789fc3 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 @@ -37,6 +37,8 @@ import io.netty.util.ReferenceCountUtil; import java.io.IOException; import java.time.Clock; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -68,6 +70,7 @@ import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.AsyncCallback.CreateCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; @@ -111,6 +114,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.ManagedLedgerMXBean; +import org.apache.bookkeeper.mledger.ManagedTrash; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.WaitingEntryCallBack; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; @@ -136,6 +140,7 @@ import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,6 +159,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { protected ManagedLedgerConfig config; protected Map propertiesMap; protected final MetaStore store; + protected final ManagedTrash managedTrash; final ConcurrentLongHashMap> ledgerCache = ConcurrentLongHashMap.>newBuilder() @@ -294,13 +300,14 @@ public enum PositionBound { Map createdLedgerCustomMetadata; public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, - ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - final String name) { - this(factory, bookKeeper, store, config, scheduledExecutor, name, null); + MetadataStore metadataStore, ManagedLedgerConfig config, + OrderedScheduler scheduledExecutor, final String name) { + this(factory, bookKeeper, store, metadataStore, config, scheduledExecutor, name, null); } public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, - ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - final String name, final Supplier mlOwnershipChecker) { + MetadataStore metadataStore, ManagedLedgerConfig config, + OrderedScheduler scheduledExecutor, final String name, + final Supplier mlOwnershipChecker) { this.factory = factory; this.bookKeeper = bookKeeper; this.config = config; @@ -333,96 +340,104 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper this.managedLedgerInterceptor = config.getManagedLedgerInterceptor(); } this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); + this.managedTrash = new ManagedTrashImpl(ManagedTrash.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, + executor, bookKeeper); } synchronized void initialize(final ManagedLedgerInitializeLedgerCallback callback, final Object ctx) { log.info("Opening managed ledger {}", name); - // Fetch the list of existing ledgers in the managed ledger - store.getManagedLedgerInfo(name, config.isCreateIfMissing(), config.getProperties(), - new MetaStoreCallback() { - @Override - public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { - ledgersStat = stat; - if (mlInfo.hasTerminatedPosition()) { - state = State.Terminated; - lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); - log.info("[{}] Recovering managed ledger terminated at {}", name, lastConfirmedEntry); - } - - for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { - ledgers.put(ls.getLedgerId(), ls); - } + managedTrash.initialize().whenComplete((res, e) -> { + if (e != null) { + callback.initializeFailed(new ManagedLedgerException(e)); + } else { + // Fetch the list of existing ledgers in the managed ledger + store.getManagedLedgerInfo(name, config.isCreateIfMissing(), config.getProperties(), + new MetaStoreCallback() { + @Override + public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { + ledgersStat = stat; + if (mlInfo.hasTerminatedPosition()) { + state = State.Terminated; + lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); + log.info("[{}] Recovering managed ledger terminated at {}", name, lastConfirmedEntry); + } - if (mlInfo.getPropertiesCount() > 0) { - propertiesMap = Maps.newHashMap(); - for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { - MLDataFormats.KeyValue property = mlInfo.getProperties(i); - propertiesMap.put(property.getKey(), property.getValue()); - } - } - if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.onManagedLedgerPropertiesInitialize(propertiesMap); - } + for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { + ledgers.put(ls.getLedgerId(), ls); + } - // Last ledger stat may be zeroed, we must update it - if (!ledgers.isEmpty()) { - final long id = ledgers.lastKey(); - OpenCallback opencb = (rc, lh, ctx1) -> { - executor.executeOrdered(name, safeRun(() -> { - mbean.endDataLedgerOpenOp(); - if (log.isDebugEnabled()) { - log.debug("[{}] Opened ledger {}: {}", name, id, BKException.getMessage(rc)); - } - if (rc == BKException.Code.OK) { - LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(id) - .setEntries(lh.getLastAddConfirmed() + 1).setSize(lh.getLength()) - .setTimestamp(clock.millis()).build(); - ledgers.put(id, info); + if (mlInfo.getPropertiesCount() > 0) { + propertiesMap = Maps.newHashMap(); + for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { + MLDataFormats.KeyValue property = mlInfo.getProperties(i); + propertiesMap.put(property.getKey(), property.getValue()); + } + } if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, lh) - .thenRun(() -> initializeBookKeeper(callback)) - .exceptionally(ex -> { - callback.initializeFailed( - new ManagedLedgerInterceptException(ex.getCause())); - return null; - }); + managedLedgerInterceptor.onManagedLedgerPropertiesInitialize(propertiesMap); + } + + // Last ledger stat may be zeroed, we must update it + if (!ledgers.isEmpty()) { + final long id = ledgers.lastKey(); + OpenCallback opencb = (rc, lh, ctx1) -> { + executor.executeOrdered(name, safeRun(() -> { + mbean.endDataLedgerOpenOp(); + if (log.isDebugEnabled()) { + log.debug("[{}] Opened ledger {}: {}", name, id, BKException.getMessage(rc)); + } + if (rc == BKException.Code.OK) { + LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(id) + .setEntries(lh.getLastAddConfirmed() + 1).setSize(lh.getLength()) + .setTimestamp(clock.millis()).build(); + ledgers.put(id, info); + if (managedLedgerInterceptor != null) { + managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, lh) + .thenRun(() -> initializeBookKeeper(callback)) + .exceptionally(ex -> { + callback.initializeFailed( + new ManagedLedgerInterceptException(ex.getCause())); + return null; + }); + } else { + initializeBookKeeper(callback); + } + } else if (isNoSuchLedgerExistsException(rc)) { + log.warn("[{}] Ledger not found: {}", name, id); + ledgers.remove(id); + initializeBookKeeper(callback); + } else { + log.error("[{}] Failed to open ledger {}: {}", name, id, BKException.getMessage(rc)); + callback.initializeFailed(createManagedLedgerException(rc)); + return; + } + })); + }; + + if (log.isDebugEnabled()) { + log.debug("[{}] Opening ledger {}", name, id); + } + mbean.startDataLedgerOpenOp(); + bookKeeper.asyncOpenLedger(id, digestType, config.getPassword(), opencb, null); } else { initializeBookKeeper(callback); } - } else if (isNoSuchLedgerExistsException(rc)) { - log.warn("[{}] Ledger not found: {}", name, id); - ledgers.remove(id); - initializeBookKeeper(callback); - } else { - log.error("[{}] Failed to open ledger {}: {}", name, id, BKException.getMessage(rc)); - callback.initializeFailed(createManagedLedgerException(rc)); - return; } - })); - }; - if (log.isDebugEnabled()) { - log.debug("[{}] Opening ledger {}", name, id); - } - mbean.startDataLedgerOpenOp(); - bookKeeper.asyncOpenLedger(id, digestType, config.getPassword(), opencb, null); - } else { - initializeBookKeeper(callback); - } - } + @Override + public void operationFailed(MetaStoreException e) { + if (e instanceof MetadataNotFoundException) { + callback.initializeFailed(new ManagedLedgerNotFoundException(e)); + } else { + callback.initializeFailed(new ManagedLedgerException(e)); + } + } + }); - @Override - public void operationFailed(MetaStoreException e) { - if (e instanceof MetadataNotFoundException) { - callback.initializeFailed(new ManagedLedgerNotFoundException(e)); - } else { - callback.initializeFailed(new ManagedLedgerException(e)); - } + scheduleTimeoutTask(); } }); - - scheduleTimeoutTask(); } private synchronized void initializeBookKeeper(final ManagedLedgerInitializeLedgerCallback callback) { @@ -2480,8 +2495,12 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { } break; } + //The offload process not completed, shouldn't delete it. + if (ls.hasOffloadContext() && !ls.getOffloadContext().getComplete()) { + continue; + } // if truncate, all ledgers besides currentLedger are going to be deleted - if (isTruncate){ + if (isTruncate) { if (log.isDebugEnabled()) { log.debug("[{}] Ledger {} will be truncated with ts {}", name, ls.getLedgerId(), ls.getTimestamp()); @@ -2542,70 +2561,107 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { advanceCursorsIfNecessary(ledgersToDelete); PositionImpl currentLastConfirmedEntry = lastConfirmedEntry; - // Update metadata - for (LedgerInfo ls : ledgersToDelete) { - if (currentLastConfirmedEntry != null && ls.getLedgerId() == currentLastConfirmedEntry.getLedgerId()) { - // this info is relevant because the lastMessageId won't be available anymore - log.info("[{}] Ledger {} contains the current last confirmed entry {}, and it is going to be " - + "deleted", name, ls.getLedgerId(), currentLastConfirmedEntry); - } - invalidateReadHandle(ls.getLedgerId()); - - ledgers.remove(ls.getLedgerId()); - NUMBER_OF_ENTRIES_UPDATER.addAndGet(this, -ls.getEntries()); - TOTAL_SIZE_UPDATER.addAndGet(this, -ls.getSize()); + // Update metadata + // Mark deletable ledgers + Set deletableLedgers = + Stream.concat(ledgersToDelete.stream().filter(ls -> !ls.getOffloadContext().getBookkeeperDeleted()), + offloadedLedgersToDelete.stream()).map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); + + // Mark deletable offloaded ledgers + Set deletableOffloadedLedgers = ledgersToDelete.stream() + .filter(ls -> ls.getOffloadContext().hasUidMsb() && ls.getOffloadContext().getComplete()) + .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); + + CompletableFuture updateTrashFuture = asyncUpdateTrashData(deletableLedgers, deletableOffloadedLedgers); + updateTrashFuture.thenAccept(ignore -> { + for (LedgerInfo ls : ledgersToDelete) { + if (currentLastConfirmedEntry != null && ls.getLedgerId() == currentLastConfirmedEntry.getLedgerId()) { + // this info is relevant because the lastMessageId won't be available anymore + log.info("[{}] Ledger {} contains the current last confirmed entry {}, and it is going to be " + + "deleted", name, ls.getLedgerId(), currentLastConfirmedEntry); + } - entryCache.invalidateAllEntries(ls.getLedgerId()); - } - for (LedgerInfo ls : offloadedLedgersToDelete) { - LedgerInfo.Builder newInfoBuilder = ls.toBuilder(); - newInfoBuilder.getOffloadContextBuilder().setBookkeeperDeleted(true); - String driverName = OffloadUtils.getOffloadDriverName(ls, - config.getLedgerOffloader().getOffloadDriverName()); - Map driverMetadata = OffloadUtils.getOffloadDriverMetadata(ls, - config.getLedgerOffloader().getOffloadDriverMetadata()); - OffloadUtils.setOffloadDriverMetadata(newInfoBuilder, driverName, driverMetadata); - ledgers.put(ls.getLedgerId(), newInfoBuilder.build()); - } + invalidateReadHandle(ls.getLedgerId()); + ledgers.remove(ls.getLedgerId()); + entryCache.invalidateAllEntries(ls.getLedgerId()); - if (log.isDebugEnabled()) { - log.debug("[{}] Updating of ledgers list after trimming", name); - } + NUMBER_OF_ENTRIES_UPDATER.addAndGet(this, -ls.getEntries()); + TOTAL_SIZE_UPDATER.addAndGet(this, -ls.getSize()); + } + for (LedgerInfo ls : offloadedLedgersToDelete) { + LedgerInfo.Builder newInfoBuilder = ls.toBuilder(); + newInfoBuilder.getOffloadContextBuilder().setBookkeeperDeleted(true); + String driverName = OffloadUtils.getOffloadDriverName(ls, + config.getLedgerOffloader().getOffloadDriverName()); + Map driverMetadata = OffloadUtils.getOffloadDriverMetadata(ls, + config.getLedgerOffloader().getOffloadDriverMetadata()); + OffloadUtils.setOffloadDriverMetadata(newInfoBuilder, driverName, driverMetadata); + ledgers.put(ls.getLedgerId(), newInfoBuilder.build()); + } - store.asyncUpdateLedgerIds(name, getManagedLedgerInfo(), ledgersStat, new MetaStoreCallback() { - @Override - public void operationComplete(Void result, Stat stat) { - log.info("[{}] End TrimConsumedLedgers. ledgers={} totalSize={}", name, ledgers.size(), - TOTAL_SIZE_UPDATER.get(ManagedLedgerImpl.this)); - ledgersStat = stat; - metadataMutex.unlock(); - trimmerMutex.unlock(); + if (log.isDebugEnabled()) { + log.debug("[{}] Updating of ledgers list after trimming", name); + } - for (LedgerInfo ls : ledgersToDelete) { - log.info("[{}] Removing ledger {} - size: {}", name, ls.getLedgerId(), ls.getSize()); - asyncDeleteLedger(ls.getLedgerId(), ls); - } - for (LedgerInfo ls : offloadedLedgersToDelete) { - log.info("[{}] Deleting offloaded ledger {} from bookkeeper - size: {}", name, ls.getLedgerId(), - ls.getSize()); - asyncDeleteLedgerFromBookKeeper(ls.getLedgerId()); + store.asyncUpdateLedgerIds(name, getManagedLedgerInfo(), ledgersStat, new MetaStoreCallback() { + @Override + public void operationComplete(Void result, Stat stat) { + log.info("[{}] End TrimConsumedLedgers. ledgers={} totalSize={}", name, ledgers.size(), + TOTAL_SIZE_UPDATER.get(ManagedLedgerImpl.this)); + ledgersStat = stat; + metadataMutex.unlock(); + trimmerMutex.unlock(); + executor.executeOrdered(name, safeRun(managedTrash::triggerDelete)); + promise.complete(null); } - promise.complete(null); - } - @Override - public void operationFailed(MetaStoreException e) { - log.warn("[{}] Failed to update the list of ledgers after trimming", name, e); - metadataMutex.unlock(); - trimmerMutex.unlock(); + @Override + public void operationFailed(MetaStoreException e) { + log.warn("[{}] Failed to update the list of ledgers after trimming", name, e); + metadataMutex.unlock(); + trimmerMutex.unlock(); - promise.completeExceptionally(e); - } + promise.completeExceptionally(e); + } + }); + }).exceptionally(ex -> { + metadataMutex.unlock(); + trimmerMutex.unlock(); + promise.completeExceptionally(ex); + return null; }); } } + private CompletableFuture asyncUpdateTrashData(Collection deletableLedgerIds, + Collection deletableOffloadedLedgerIds) { + List> futures = + new ArrayList<>(deletableLedgerIds.size() + deletableOffloadedLedgerIds.size()); + for (Long ledgerId : deletableLedgerIds) { + futures.add(managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.DELETABLE_LEDGER_SUFFIX)); + } + for (Long ledgerId : deletableOffloadedLedgerIds) { + futures.add(managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), + ManagedTrash.DELETABLE_OFFLOADED_LEDGER_SUFFIX)); + } + + CompletableFuture future = new CompletableFuture<>(); + futures.add(future); + managedTrash.asyncUpdateTrashData(Optional.of(new ManagedTrash.TrashMetaStoreCallback() { + @Override + public void operationComplete(Void result) { + future.complete(null); + } + + @Override + public void operationFailed(MetaStoreException e) { + future.completeExceptionally(e); + } + })); + return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])); + } + /** * Non-durable cursors have to be moved forward when data is trimmed since they are not retain that data. * This method also addresses a corner case for durable cursors in which the cursor is caught up, i.e. the mark @@ -2737,25 +2793,6 @@ public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) { } } - private void asyncDeleteLedgerFromBookKeeper(long ledgerId) { - asyncDeleteLedger(ledgerId, DEFAULT_LEDGER_DELETE_RETRIES); - } - - private void asyncDeleteLedger(long ledgerId, LedgerInfo info) { - if (!info.getOffloadContext().getBookkeeperDeleted()) { - // only delete if it hasn't been previously deleted for offload - asyncDeleteLedger(ledgerId, DEFAULT_LEDGER_DELETE_RETRIES); - } - - if (info.getOffloadContext().hasUidMsb()) { - UUID uuid = new UUID(info.getOffloadContext().getUidMsb(), info.getOffloadContext().getUidLsb()); - cleanupOffloaded(ledgerId, uuid, - OffloadUtils.getOffloadDriverName(info, config.getLedgerOffloader().getOffloadDriverName()), - OffloadUtils.getOffloadDriverMetadata(info, config.getLedgerOffloader().getOffloadDriverMetadata()), - "Trimming"); - } - } - private void asyncDeleteLedger(long ledgerId, long retry) { if (retry <= 0) { log.warn("[{}] Failed to delete ledger after retries {}", name, ledgerId); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java new file mode 100644 index 0000000000000..0c6442100ac60 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -0,0 +1,619 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import static org.apache.bookkeeper.mledger.util.Errors.isNoSuchLedgerExistsException; +import static org.apache.bookkeeper.mledger.util.SafeRun.safeRun; +import com.google.common.collect.Maps; +import com.google.protobuf.InvalidProtocolBufferException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedTrash; +import org.apache.bookkeeper.mledger.ManagedTrashMXBean; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.TrashDataComponent; +import org.apache.bookkeeper.mledger.util.CallbackMutex; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.Stat; +import org.apache.pulsar.metadata.api.extended.CreateOption; +import org.apache.pulsar.metadata.impl.batching.AbstractBatchedMetadataStore; +import org.apache.pulsar.metadata.impl.batching.MetadataOp; +import org.apache.pulsar.metadata.impl.batching.OpPut; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ManagedTrashImpl implements ManagedTrash { + + private static final Logger log = LoggerFactory.getLogger(ManagedTrashImpl.class); + + private static final String BASE_NODE = "/trash-data"; + + private static final String PREFIX = BASE_NODE + "/"; + + private static final String DELETE = "/delete"; + + private static final String ARCHIVE = "/archive-"; + + private static final String TRASH_KEY_SEPARATOR = "-"; + + private static final int RETRY_COUNT = 9; + + private static final CompletableFuture COMPLETED_FUTURE = CompletableFuture.completedFuture(null); + + //key:ledgerId value:storageContext + private NavigableMap trashData = new ConcurrentSkipListMap<>(); + + private final AtomicInteger toArchiveCount = new AtomicInteger(); + + private final CallbackMutex trashMutex = new CallbackMutex(); + + private final CallbackMutex deleteMutex = new CallbackMutex(); + + private final CallbackMutex trashPersistMutex = new CallbackMutex(); + + private final AbstractBatchedMetadataStore metadataStore; + + private volatile Stat deleteStat; + + private final String type; + + private final String name; + + private final ManagedLedgerConfig config; + + private final OrderedScheduler scheduledExecutor; + + private final OrderedExecutor executor; + + private final BookKeeper bookKeeper; + + private final int trashDataLimitSize; + + private volatile boolean trashIsDirty; + + private ScheduledFuture checkTrashPersistTask; + + private ScheduledFuture triggerDeleteTask; + + private final ManagedTrashMXBean managedTrashMXBean; + + public ManagedTrashImpl(String type, String name, MetadataStore metadataStore, ManagedLedgerConfig config, + OrderedScheduler scheduledExecutor, OrderedExecutor executor, BookKeeper bookKeeper) { + this.type = type; + this.name = name; + this.config = config; + if (!(metadataStore instanceof AbstractBatchedMetadataStore)) { + throw new IllegalStateException("ManagedTrashImpl metadata store must support batch operation."); + } + this.metadataStore = (AbstractBatchedMetadataStore) metadataStore; + this.scheduledExecutor = scheduledExecutor; + this.executor = executor; + this.bookKeeper = bookKeeper; + this.trashDataLimitSize = config.getTrashDataLimitSize(); + this.managedTrashMXBean = new ManagedTrashMXBeanImpl(this); + } + + @Override + public String name() { + return name + "-" + type; + } + + @Override + public CompletableFuture initialize() { + CompletableFuture future = new CompletableFuture<>(); + metadataStore.get(buildDeletePath()).whenCompleteAsync((res, e) -> { + if (e != null) { + log.error("Get delete data failed, name:{} type: {}", name, type, e); + future.completeExceptionally(e); + } else { + if (res.isEmpty()) { + future.complete(null); + return; + } + byte[] value = res.get().getValue(); + try { + trashData.putAll(deSerialize(value)); + deleteStat = res.get().getStat(); + toArchiveCount.set(calculateArchiveCount().get()); + future.complete(null); + checkTrashPersistTask = + scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, + TimeUnit.MINUTES); + triggerDeleteTask = + scheduledExecutor.scheduleAtFixedRate(safeRun(this::triggerDeleteInBackground), 1L, 1L, + TimeUnit.MINUTES); + } catch (InvalidProtocolBufferException exc) { + future.completeExceptionally(getException(exc)); + } catch (ExecutionException | InterruptedException ex) { + ex.printStackTrace(); + } + } + }, scheduledExecutor.chooseThread(name)); + return future; + } + + private void persistTrashIfNecessary() { + if (trashIsDirty) { + asyncUpdateTrashData(Optional.empty()); + } + } + + private CompletableFuture calculateArchiveCount() { + CompletableFuture future = new CompletableFuture<>(); + internalCalculateArchiveCount(future); + return future; + } + + private void internalCalculateArchiveCount(CompletableFuture future) { + if (!trashMutex.tryLock()) { + scheduledExecutor.schedule(safeRun(() -> internalCalculateArchiveCount(future)), 100, + TimeUnit.MILLISECONDS); + return; + } + try { + int toArchiveCount = 0; + for (Map.Entry entry : trashData.entrySet()) { + if (!entry.getKey().startsWith("0")) { + break; + } + toArchiveCount++; + } + future.complete(toArchiveCount); + } finally { + trashMutex.unlock(); + } + } + + @Override + public CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) { + CompletableFuture future = new CompletableFuture<>(); + appendTrashData(buildKey(RETRY_COUNT, ledgerId, type), context, future); + return future; + } + + private static ManagedLedgerException.MetaStoreException getException(Throwable t) { + if (t.getCause() instanceof MetadataStoreException.BadVersionException) { + return new ManagedLedgerException.BadVersionException(t.getMessage()); + } else { + return new ManagedLedgerException.MetaStoreException(t); + } + } + + public void appendInBackground(final String key, final LedgerInfo context, final CompletableFuture future) { + executor.executeOrdered(name, safeRun(() -> appendTrashData(key, context, future))); + } + + private void appendTrashData(final String key, final LedgerInfo context, final CompletableFuture future) { + if (!trashMutex.tryLock()) { + scheduledExecutor.schedule(safeRun(() -> appendInBackground(key, context, future)), 100, + TimeUnit.MILLISECONDS); + return; + } + try { + trashData.put(key, context); + trashIsDirty = true; + } finally { + trashMutex.unlock(); + } + } + + @Override + public void asyncUpdateTrashData(Optional> callback) { + metadataStore.put(buildDeletePath(), serialize(trashData), + deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion())) + .whenCompleteAsync((res, e) -> { + if (e != null) { + callback.ifPresent(call -> call.operationFailed(getException(e))); + return; + } + deleteStat = res; + trashIsDirty = false; + trashPersistMutex.unlock(); + callback.ifPresent(call -> call.operationComplete(null)); + }, executor.chooseThread(name)); + } + + private byte[] serialize(Map toPersist) { + TrashDataComponent.Builder builder = TrashDataComponent.newBuilder(); + for (Map.Entry entry : toPersist.entrySet()) { + builder.addComponent( + MLDataFormats.TrashData.newBuilder().setKey(entry.getKey()).setValue(entry.getValue()).build()); + } + return builder.build().toByteArray(); + } + + private Map deSerialize(byte[] content) throws InvalidProtocolBufferException { + TrashDataComponent component = TrashDataComponent.parseFrom(content); + List componentList = component.getComponentList(); + Map result = new HashMap<>(); + for (MLDataFormats.TrashData ele : componentList) { + result.put(ele.getKey(), ele.getValue()); + } + return result; + } + + private void triggerDeleteInBackground() { + executor.executeOrdered(name, safeRun(this::triggerDelete)); + } + + @Override + public void triggerDelete() { + if (!deleteMutex.tryLock()) { + scheduledExecutor.schedule(this::triggerDeleteInBackground, 100, TimeUnit.MILLISECONDS); + return; + } + if (!trashMutex.tryLock()) { + deleteMutex.unlock(); + scheduledExecutor.schedule(this::triggerDeleteInBackground, 100, TimeUnit.MILLISECONDS); + return; + } + List toDelete = getToDeleteData(); + if (toDelete.size() == 0) { + deleteMutex.unlock(); + trashMutex.unlock(); + return; + } + for (TrashDeleteHelper delHelper : toDelete) { + //unlock in asyncDeleteTrash + asyncDeleteTrash(delHelper); + } + } + + @Override + public List getAllArchiveIndex() { + return null; + } + + @Override + public Map getArchiveData(final long index) { + return null; + } + + @Override + public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { + if (checkTrashPersistTask != null) { + checkTrashPersistTask.cancel(true); + } + if (triggerDeleteTask != null) { + triggerDeleteTask.cancel(true); + } + asyncUpdateArchiveData(new TrashMetaStoreCallback<>() { + @Override + public void operationComplete(Void result) { + callback.closeComplete(ctx); + } + + @Override + public void operationFailed(ManagedLedgerException.MetaStoreException e) { + callback.closeFailed(getException(e), ctx); + } + }); + } + + private void increaseArchiveCountWhenDeleteFailed(final CompletableFuture future) { + toArchiveCount.incrementAndGet(); + updateArchiveDataIfNecessary(future); + } + + private void updateArchiveDataIfNecessary(final CompletableFuture future) { + if (toArchiveCount.get() < trashDataLimitSize / 2) { + future.complete(null); + return; + } + asyncUpdateArchiveData(new TrashMetaStoreCallback<>() { + @Override + public void operationComplete(Void result) { + future.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException.MetaStoreException e) { + future.completeExceptionally(e); + } + }); + } + + + private String buildPath() { + return PREFIX + name + "/" + type; + } + + private String buildDeletePath() { + return buildPath() + DELETE; + } + + private String buildArchivePath(long ledgerId) { + return buildPath() + ARCHIVE + ledgerId; + } + + //take 1/10 trash to delete, if the size over 10, use 10 to delete. + private List getToDeleteData() { + if (trashData.size() == 0) { + return Collections.emptyList(); + } + int batchSize = trashData.size() / 10; + if (batchSize > 10) { + batchSize = 10; + } + if (batchSize == 0) { + batchSize = 1; + } + List toDelete = new ArrayList<>(batchSize); + for (Map.Entry entry : trashData.descendingMap().entrySet()) { + TrashDeleteHelper delHelper = TrashDeleteHelper.build(entry.getKey(), entry.getValue()); + //if last retryCount is zero, the before data retryCount is zero too. + if (delHelper.retryCount == 0) { + break; + } + toDelete.add(delHelper); + if (toDelete.size() == batchSize) { + break; + } + } + return toDelete; + } + + private void asyncUpdateArchiveData(ManagedTrash.TrashMetaStoreCallback callback) { + //transaction operation + NavigableMap persistDelete = new ConcurrentSkipListMap<>(); + NavigableMap persistArchive = new ConcurrentSkipListMap<>(); + + + for (Map.Entry entry : trashData.entrySet()) { + persistArchive.put(entry.getKey(), entry.getValue()); + if (persistArchive.size() >= trashDataLimitSize / 2) { + break; + } + } + + persistDelete.putAll(trashData); + for (Map.Entry entry : persistArchive.entrySet()) { + persistDelete.remove(entry.getKey()); + } + //build delete persist operation + List txOps = new ArrayList<>(2); + OpPut opDeletePersist = new OpPut(buildDeletePath(), serialize(persistDelete), + deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion()), + EnumSet.noneOf(CreateOption.class)); + //build archive persist operation + Map.Entry lastEntry = persistArchive.lastEntry(); + OpPut opArchivePersist = + new OpPut(buildArchivePath(TrashDeleteHelper.build(lastEntry.getKey(), lastEntry.getValue()).ledgerId), + serialize(persistArchive), Optional.of(-1L), EnumSet.noneOf(CreateOption.class)); + txOps.add(opDeletePersist); + txOps.add(opArchivePersist); + metadataStore.batchOperation(txOps); + + opDeletePersist.getFuture().whenCompleteAsync((res, e) -> { + if (e != null) { + log.error("Persist trash data failed.", e); + callback.operationFailed(getException(e)); + return; + } + opArchivePersist.getFuture().whenComplete((res1, e1) -> { + if (e1 != null) { + log.error("Persist archive data failed.", e1); + callback.operationFailed(getException(e1)); + return; + } + deleteStat = res; + trashData = persistDelete; + trashIsDirty = false; + toArchiveCount.set(0); + }); + }, executor.chooseThread(name)); + + } + + private static String buildKey(int retryCount, long ledgerId, String suffix) { + return retryCount + TRASH_KEY_SEPARATOR + String.format("%019d", ledgerId) + TRASH_KEY_SEPARATOR + + suffix; + } + + private void asyncDeleteTrash(TrashDeleteHelper delHelper) { + if (delHelper.isLedger()) { + asyncDeleteLedger(delHelper.ledgerId, new AsyncCallbacks.DeleteLedgerCallback() { + @Override + public void deleteLedgerComplete(Object ctx) { + onDeleteSuccess(delHelper); + } + + @Override + public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { + onDeleteFailed(delHelper); + } + }); + } else if (delHelper.isOffloadLedger()) { + asyncDeleteOffloadedLedger(delHelper.ledgerId, delHelper.context, + new AsyncCallbacks.DeleteLedgerCallback() { + @Override + public void deleteLedgerComplete(Object ctx) { + onDeleteSuccess(delHelper); + } + + @Override + public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { + onDeleteFailed(delHelper); + } + }); + } + } + + private void onDeleteSuccess(TrashDeleteHelper delHelper) { + try { + String key = delHelper.transferToTrashKey(); + if (log.isDebugEnabled()) { + String info = null; + if (delHelper.isLedger()) { + info = String.format("Delete ledger %s success.", delHelper.ledgerId); + } else if (delHelper.isOffloadLedger()) { + info = String.format("Delete offload ledger %s success.", delHelper.ledgerId); + } + log.debug(info); + } + trashData.remove(key); + trashIsDirty = true; + } finally { + deleteMutex.unlock(); + trashMutex.unlock(); + } + } + + private void onDeleteFailed(TrashDeleteHelper delHelper) { + try { + //override old key + String key = delHelper.transferToTrashKey(); + trashData.remove(key); + trashData.put(buildKey(delHelper.retryCount - 1, delHelper.ledgerId, delHelper.suffix), + delHelper.context); + trashIsDirty = true; + if (delHelper.retryCount - 1 == 0) { + if (log.isDebugEnabled()) { + String info = null; + if (delHelper.isLedger()) { + info = String.format("Delete ledger %d reach retry limit %d.", delHelper.ledgerId, RETRY_COUNT); + } else if (delHelper.isOffloadLedger()) { + info = String.format("Delete offload ledger %d reach retry limit %d.", delHelper.ledgerId, + RETRY_COUNT); + } + log.debug(info); + } + increaseArchiveCountWhenDeleteFailed(COMPLETED_FUTURE); + } + } finally { + deleteMutex.unlock(); + trashMutex.unlock(); + } + } + + private void asyncDeleteLedger(long ledgerId, AsyncCallbacks.DeleteLedgerCallback callback) { + log.info("[{}] Removing ledger {}", name, ledgerId); + bookKeeper.asyncDeleteLedger(ledgerId, (rc, ctx) -> { + if (isNoSuchLedgerExistsException(rc)) { + log.warn("[{}] Ledger was already deleted {}", name, ledgerId); + } else if (rc != BKException.Code.OK) { + log.error("[{}] Error delete ledger {} : {}", name, ledgerId, BKException.getMessage(rc)); + callback.deleteLedgerFailed(ManagedLedgerImpl.createManagedLedgerException(rc), null); + return; + } + if (log.isDebugEnabled()) { + log.debug("[{}] Deleted ledger {}", name, ledgerId); + } + callback.deleteLedgerComplete(ctx); + }, null); + } + + private void asyncDeleteOffloadedLedger(long ledgerId, LedgerInfo info, + AsyncCallbacks.DeleteLedgerCallback callback) { + if (!info.getOffloadContext().hasUidMsb()) { + return; + } + String cleanupReason = "Trash-Trimming"; + + UUID uuid = new UUID(info.getOffloadContext().getUidMsb(), info.getOffloadContext().getUidLsb()); + + log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.", name, ledgerId, uuid, + cleanupReason); + Map metadataMap = Maps.newHashMap(); + metadataMap.putAll(config.getLedgerOffloader().getOffloadDriverMetadata()); + metadataMap.put("ManagedLedgerName", name); + + try { + config.getLedgerOffloader() + .deleteOffloaded(ledgerId, uuid, metadataMap) + .whenComplete((ignored, exception) -> { + if (exception != null) { + log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})", + name, ledgerId, cleanupReason, exception); + log.warn("[{}] Failed to delete offloaded ledger after retries {} / {}", name, ledgerId, + uuid); + callback.deleteLedgerFailed( + new ManagedLedgerException("Failed to delete offloaded ledger after retries"), + null); + return; + } + callback.deleteLedgerComplete(null); + }); + } catch (Exception e) { + log.warn("[{}] Failed to cleanup offloaded ledgers.", name, e); + } + } + + + private static class TrashDeleteHelper { + + private final int retryCount; + + private final long ledgerId; + + private final String suffix; + + private final LedgerInfo context; + + public TrashDeleteHelper(int retryCount, long ledgerId, String suffix, LedgerInfo context) { + this.retryCount = retryCount; + this.ledgerId = ledgerId; + this.suffix = suffix; + this.context = context; + } + + public static TrashDeleteHelper build(String key, LedgerInfo context) { + String[] split = key.split(TRASH_KEY_SEPARATOR); + int retryCont = Integer.parseInt(split[0]); + long ledgerId = Long.parseLong(split[1]); + return new TrashDeleteHelper(retryCont, ledgerId, split[2], context); + } + + private String transferToTrashKey() { + return ManagedTrashImpl.buildKey(retryCount, ledgerId, suffix); + } + + private boolean isLedger() { + return DELETABLE_LEDGER_SUFFIX.equals(suffix); + } + + private boolean isOffloadLedger() { + return DELETABLE_OFFLOADED_LEDGER_SUFFIX.equals(suffix); + } + } + + +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java new file mode 100644 index 0000000000000..4949749417000 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import org.apache.bookkeeper.mledger.ManagedTrash; +import org.apache.bookkeeper.mledger.ManagedTrashMXBean; + +public class ManagedTrashMXBeanImpl implements ManagedTrashMXBean { + + private final ManagedTrash managedTrash; + + public ManagedTrashMXBeanImpl(ManagedTrash managedTrash) { + this.managedTrash = managedTrash; + } + + @Override + public String getName() { + return managedTrash.name(); + } + + @Override + public String getType() { + return null; + } + + @Override + public long getCurrentNumberOfLedgersWaitingToDeleted() { + return managedTrash.; + } + + @Override + public long getTotalNumberOfLedgersWaitingToDeleted() { + return ; + } + + @Override + public long getCurrentNumberOfLedgersWaitingToArchive() { + return 0; + } + + @Override + public long getTotalNumberOfLedgersWaitingToArchive() { + return 0; + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 214c3afe1bcdb..d3c4d6baa615d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -34,15 +34,16 @@ import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Stat; @Slf4j public class ReadOnlyManagedLedgerImpl extends ManagedLedgerImpl { public ReadOnlyManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, - ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - String name) { - super(factory, bookKeeper, store, config, scheduledExecutor, name); + MetadataStore metadataStore, ManagedLedgerConfig config, + OrderedScheduler scheduledExecutor, String name) { + super(factory, bookKeeper, store, metadataStore, config, scheduledExecutor, name); } CompletableFuture initializeAndCreateCursor(PositionImpl startPosition) { diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index 4671816c1a199..b0a025b95472c 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -142,3 +142,13 @@ message ManagedCursorInfoMetadata { required CompressionType compressionType = 1; required int32 uncompressedSize = 2; } + +message TrashDataComponent { + repeated TrashData component = 1; +} + +message TrashData { + required string key = 1; + optional ManagedLedgerInfo.LedgerInfo value = 2; +} + diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java index e1c26bead096a..707410cb2ccae 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java @@ -159,7 +159,7 @@ protected CompletableFuture storePut(String path, byte[] data, Optional ops) { + public void batchOperation(List ops) { try { Txn txn = kv.txn(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 6697934d56b3a..badb45f189d3f 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -149,7 +149,7 @@ protected void receivedSessionEvent(SessionEvent event) { } @Override - protected void batchOperation(List ops) { + public void batchOperation(List ops) { try { zkc.multi(ops.stream().map(this::convertOp).collect(Collectors.toList()), (rc, path, ctx, results) -> { if (results == null) { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java index 616cac289efea..dd17cd8f5512b 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java @@ -158,5 +158,5 @@ private void enqueue(MessagePassingQueue queue, MetadataOp op) { } } - protected abstract void batchOperation(List ops); + public abstract void batchOperation(List ops); } From df055a6e2f8f5f5981b2a666a953d17534123546 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Sat, 28 May 2022 16:14:07 +0800 Subject: [PATCH 02/43] redefine TrashDataComponent proto --- conf/broker.conf | 18 +- .../mledger/ManagedLedgerConfig.java | 8 + .../bookkeeper/mledger/ManagedTrash.java | 61 ++ .../mledger/ManagedTrashMXBean.java | 44 ++ .../impl/ManagedLedgerFactoryImpl.java | 4 +- .../mledger/impl/ManagedLedgerImpl.java | 339 +++++----- .../mledger/impl/ManagedTrashImpl.java | 619 ++++++++++++++++++ .../mledger/impl/ManagedTrashMXBeanImpl.java | 61 ++ .../impl/ReadOnlyManagedLedgerImpl.java | 7 +- .../src/main/proto/MLDataFormats.proto | 10 + .../metadata/impl/EtcdMetadataStore.java | 2 +- .../pulsar/metadata/impl/ZKMetadataStore.java | 2 +- .../AbstractBatchedMetadataStore.java | 2 +- 13 files changed, 1009 insertions(+), 168 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java diff --git a/conf/broker.conf b/conf/broker.conf index ee1eb3a7a6323..1f9b3389a238b 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -109,7 +109,7 @@ enableBusyWait=false isRunningStandalone= # Name of the cluster to which this broker belongs to -clusterName= +clusterName=horizon-cluster # The maximum number of tenants that each pulsar cluster can create # This configuration is not precise control, in a concurrent scenario, the threshold will be exceeded @@ -119,7 +119,7 @@ maxTenants=0 failureDomainsEnabled=false # Metadata store session timeout in milliseconds -metadataStoreSessionTimeoutMillis=30000 +metadataStoreSessionTimeoutMillis=600000 # Metadata store operation timeout in seconds metadataStoreOperationTimeoutSeconds=30 @@ -232,7 +232,7 @@ subscriptionKeySharedConsistentHashingReplicaPoints=100 # Set the default behavior for message deduplication in the broker # This can be overridden per-namespace. If enabled, broker will reject # messages that were already stored in the topic -brokerDeduplicationEnabled=false +brokerDeduplicationEnabled=true # Maximum number of producer information that it's going to be # persisted for deduplication purposes @@ -515,7 +515,7 @@ delayedDeliveryEnabled=true delayedDeliveryTickTimeMillis=1000 # Whether to enable acknowledge of batch local index. -acknowledgmentAtBatchIndexLevelEnabled=false +acknowledgmentAtBatchIndexLevelEnabled=true # Enable tracking of replicated subscriptions state across clusters. enableReplicatedSubscriptions=true @@ -810,7 +810,7 @@ maxConcurrentHttpRequests=1024 # For example: zk+hierarchical://localhost:2181/ledgers # The metadata service uri list can also be semicolon separated values like below: # zk+hierarchical://zk1:2181;zk2:2181;zk3:2181/ledgers -bookkeeperMetadataServiceUri= +bookkeeperMetadataServiceUri=zk+hierarchical://127.0.0.1:2181/ledgers # Authentication plugin to use when connecting to bookies bookkeeperClientAuthenticationPlugin= @@ -941,10 +941,10 @@ bookkeeperExplicitLacIntervalInMills=0 managedLedgerDefaultEnsembleSize=2 # Number of copies to store for each message -managedLedgerDefaultWriteQuorum=2 +managedLedgerDefaultWriteQuorum=1 # Number of guaranteed copies (acks to wait before write is complete) -managedLedgerDefaultAckQuorum=2 +managedLedgerDefaultAckQuorum=1 # with OpportunisticStriping=true the ensembleSize is adapted automatically to writeQuorum # in case of lack of enough bookies @@ -1440,12 +1440,12 @@ strictBookieAffinityEnabled=false # Zookeeper quorum connection string # Deprecated: use metadataStoreUrl instead -zookeeperServers= +zookeeperServers=127.0.0.1:2181 # Configuration Store connection string # Deprecated. Use configurationMetadataStoreUrl globalZookeeperServers= -configurationStoreServers= +configurationStoreServers=127.0.0.1:2181 # Zookeeper session timeout in milliseconds # Deprecated: use metadataStoreSessionTimeoutMillis diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index e628a253563a1..83723ae9fcf24 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -81,6 +81,7 @@ public class ManagedLedgerConfig { @Getter @Setter private boolean cacheEvictionByMarkDeletedPosition = false; + private int trashDataLimitSize = 1000; public boolean isCreateIfMissing() { return createIfMissing; @@ -683,4 +684,11 @@ public void setInactiveLedgerRollOverTime(int inactiveLedgerRollOverTimeMs, Time this.inactiveLedgerRollOverTimeMs = (int) unit.toMillis(inactiveLedgerRollOverTimeMs); } + public int getTrashDataLimitSize() { + return trashDataLimitSize; + } + + public void setTrashDataLimitSize(int trashDataLimitSize) { + this.trashDataLimitSize = trashDataLimitSize; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java new file mode 100644 index 0000000000000..64e38f93e95d1 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; + +public interface ManagedTrash { + + String MANAGED_LEDGER = "managed-ledger"; + + String MANAGED_CURSOR = "managed-cursor"; + + String SCHEMA = "schema"; + + String DELETABLE_LEDGER_SUFFIX = "DL"; + + String DELETABLE_OFFLOADED_LEDGER_SUFFIX = "DOL"; + + interface TrashMetaStoreCallback { + + void operationComplete(T result); + + void operationFailed(ManagedLedgerException.MetaStoreException e); + } + + String name(); + + CompletableFuture initialize(); + + CompletableFuture appendLedgerTrashData(final long ledgerId, final LedgerInfo context, final String type); + + void asyncUpdateTrashData(Optional> callback); + + void triggerDelete(); + + List getAllArchiveIndex(); + + Map getArchiveData(final long index); + + void asyncClose(final AsyncCallbacks.CloseCallback callback, final Object ctx); +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java new file mode 100644 index 0000000000000..34ac52b99f2bd --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger; + +import org.apache.bookkeeper.common.annotation.InterfaceAudience; +import org.apache.bookkeeper.common.annotation.InterfaceStability; + +/** + * JMX Bean interface for ManagedTrash stats. + */ +@InterfaceAudience.LimitedPrivate +@InterfaceStability.Stable +public interface ManagedTrashMXBean { + + String getName(); + + long getCurrentNumberOfLedgersWaitingToDeleted(); + + long getTotalNumberOfLedgersWaitingToDeleted(); + + void + + long getCurrentNumberOfLedgersWaitingToArchive(); + + long getTotalNumberOfLedgersWaitingToArchive(); + + +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index b6b1bcf3e8d9a..bacee90d9dba1 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -383,7 +383,7 @@ public void asyncOpen(final String name, final ManagedLedgerConfig config, final bookkeeperFactory.get( new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), config.getBookKeeperEnsemblePlacementPolicyProperties())), - store, config, scheduledExecutor, name, mlOwnershipChecker); + store, metadataStore, config, scheduledExecutor, name, mlOwnershipChecker); PendingInitializeManagedLedger pendingLedger = new PendingInitializeManagedLedger(newledger); pendingInitializeLedgers.put(name, pendingLedger); newledger.initialize(new ManagedLedgerInitializeLedgerCallback() { @@ -476,7 +476,7 @@ public void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosi bookkeeperFactory .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), config.getBookKeeperEnsemblePlacementPolicyProperties())), - store, config, scheduledExecutor, managedLedgerName); + store, metadataStore, config, scheduledExecutor, managedLedgerName); roManagedLedger.initializeAndCreateCursor((PositionImpl) startPosition) .thenAccept(roCursor -> callback.openReadOnlyCursorComplete(roCursor, ctx)) 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 277e4df0244b8..67ff37a789fc3 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 @@ -37,6 +37,8 @@ import io.netty.util.ReferenceCountUtil; import java.io.IOException; import java.time.Clock; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -68,6 +70,7 @@ import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.AsyncCallback.CreateCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; @@ -111,6 +114,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.ManagedLedgerMXBean; +import org.apache.bookkeeper.mledger.ManagedTrash; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.WaitingEntryCallBack; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; @@ -136,6 +140,7 @@ import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,6 +159,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { protected ManagedLedgerConfig config; protected Map propertiesMap; protected final MetaStore store; + protected final ManagedTrash managedTrash; final ConcurrentLongHashMap> ledgerCache = ConcurrentLongHashMap.>newBuilder() @@ -294,13 +300,14 @@ public enum PositionBound { Map createdLedgerCustomMetadata; public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, - ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - final String name) { - this(factory, bookKeeper, store, config, scheduledExecutor, name, null); + MetadataStore metadataStore, ManagedLedgerConfig config, + OrderedScheduler scheduledExecutor, final String name) { + this(factory, bookKeeper, store, metadataStore, config, scheduledExecutor, name, null); } public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, - ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - final String name, final Supplier mlOwnershipChecker) { + MetadataStore metadataStore, ManagedLedgerConfig config, + OrderedScheduler scheduledExecutor, final String name, + final Supplier mlOwnershipChecker) { this.factory = factory; this.bookKeeper = bookKeeper; this.config = config; @@ -333,96 +340,104 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper this.managedLedgerInterceptor = config.getManagedLedgerInterceptor(); } this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); + this.managedTrash = new ManagedTrashImpl(ManagedTrash.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, + executor, bookKeeper); } synchronized void initialize(final ManagedLedgerInitializeLedgerCallback callback, final Object ctx) { log.info("Opening managed ledger {}", name); - // Fetch the list of existing ledgers in the managed ledger - store.getManagedLedgerInfo(name, config.isCreateIfMissing(), config.getProperties(), - new MetaStoreCallback() { - @Override - public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { - ledgersStat = stat; - if (mlInfo.hasTerminatedPosition()) { - state = State.Terminated; - lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); - log.info("[{}] Recovering managed ledger terminated at {}", name, lastConfirmedEntry); - } - - for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { - ledgers.put(ls.getLedgerId(), ls); - } + managedTrash.initialize().whenComplete((res, e) -> { + if (e != null) { + callback.initializeFailed(new ManagedLedgerException(e)); + } else { + // Fetch the list of existing ledgers in the managed ledger + store.getManagedLedgerInfo(name, config.isCreateIfMissing(), config.getProperties(), + new MetaStoreCallback() { + @Override + public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { + ledgersStat = stat; + if (mlInfo.hasTerminatedPosition()) { + state = State.Terminated; + lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); + log.info("[{}] Recovering managed ledger terminated at {}", name, lastConfirmedEntry); + } - if (mlInfo.getPropertiesCount() > 0) { - propertiesMap = Maps.newHashMap(); - for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { - MLDataFormats.KeyValue property = mlInfo.getProperties(i); - propertiesMap.put(property.getKey(), property.getValue()); - } - } - if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.onManagedLedgerPropertiesInitialize(propertiesMap); - } + for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { + ledgers.put(ls.getLedgerId(), ls); + } - // Last ledger stat may be zeroed, we must update it - if (!ledgers.isEmpty()) { - final long id = ledgers.lastKey(); - OpenCallback opencb = (rc, lh, ctx1) -> { - executor.executeOrdered(name, safeRun(() -> { - mbean.endDataLedgerOpenOp(); - if (log.isDebugEnabled()) { - log.debug("[{}] Opened ledger {}: {}", name, id, BKException.getMessage(rc)); - } - if (rc == BKException.Code.OK) { - LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(id) - .setEntries(lh.getLastAddConfirmed() + 1).setSize(lh.getLength()) - .setTimestamp(clock.millis()).build(); - ledgers.put(id, info); + if (mlInfo.getPropertiesCount() > 0) { + propertiesMap = Maps.newHashMap(); + for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { + MLDataFormats.KeyValue property = mlInfo.getProperties(i); + propertiesMap.put(property.getKey(), property.getValue()); + } + } if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, lh) - .thenRun(() -> initializeBookKeeper(callback)) - .exceptionally(ex -> { - callback.initializeFailed( - new ManagedLedgerInterceptException(ex.getCause())); - return null; - }); + managedLedgerInterceptor.onManagedLedgerPropertiesInitialize(propertiesMap); + } + + // Last ledger stat may be zeroed, we must update it + if (!ledgers.isEmpty()) { + final long id = ledgers.lastKey(); + OpenCallback opencb = (rc, lh, ctx1) -> { + executor.executeOrdered(name, safeRun(() -> { + mbean.endDataLedgerOpenOp(); + if (log.isDebugEnabled()) { + log.debug("[{}] Opened ledger {}: {}", name, id, BKException.getMessage(rc)); + } + if (rc == BKException.Code.OK) { + LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(id) + .setEntries(lh.getLastAddConfirmed() + 1).setSize(lh.getLength()) + .setTimestamp(clock.millis()).build(); + ledgers.put(id, info); + if (managedLedgerInterceptor != null) { + managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, lh) + .thenRun(() -> initializeBookKeeper(callback)) + .exceptionally(ex -> { + callback.initializeFailed( + new ManagedLedgerInterceptException(ex.getCause())); + return null; + }); + } else { + initializeBookKeeper(callback); + } + } else if (isNoSuchLedgerExistsException(rc)) { + log.warn("[{}] Ledger not found: {}", name, id); + ledgers.remove(id); + initializeBookKeeper(callback); + } else { + log.error("[{}] Failed to open ledger {}: {}", name, id, BKException.getMessage(rc)); + callback.initializeFailed(createManagedLedgerException(rc)); + return; + } + })); + }; + + if (log.isDebugEnabled()) { + log.debug("[{}] Opening ledger {}", name, id); + } + mbean.startDataLedgerOpenOp(); + bookKeeper.asyncOpenLedger(id, digestType, config.getPassword(), opencb, null); } else { initializeBookKeeper(callback); } - } else if (isNoSuchLedgerExistsException(rc)) { - log.warn("[{}] Ledger not found: {}", name, id); - ledgers.remove(id); - initializeBookKeeper(callback); - } else { - log.error("[{}] Failed to open ledger {}: {}", name, id, BKException.getMessage(rc)); - callback.initializeFailed(createManagedLedgerException(rc)); - return; } - })); - }; - if (log.isDebugEnabled()) { - log.debug("[{}] Opening ledger {}", name, id); - } - mbean.startDataLedgerOpenOp(); - bookKeeper.asyncOpenLedger(id, digestType, config.getPassword(), opencb, null); - } else { - initializeBookKeeper(callback); - } - } + @Override + public void operationFailed(MetaStoreException e) { + if (e instanceof MetadataNotFoundException) { + callback.initializeFailed(new ManagedLedgerNotFoundException(e)); + } else { + callback.initializeFailed(new ManagedLedgerException(e)); + } + } + }); - @Override - public void operationFailed(MetaStoreException e) { - if (e instanceof MetadataNotFoundException) { - callback.initializeFailed(new ManagedLedgerNotFoundException(e)); - } else { - callback.initializeFailed(new ManagedLedgerException(e)); - } + scheduleTimeoutTask(); } }); - - scheduleTimeoutTask(); } private synchronized void initializeBookKeeper(final ManagedLedgerInitializeLedgerCallback callback) { @@ -2480,8 +2495,12 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { } break; } + //The offload process not completed, shouldn't delete it. + if (ls.hasOffloadContext() && !ls.getOffloadContext().getComplete()) { + continue; + } // if truncate, all ledgers besides currentLedger are going to be deleted - if (isTruncate){ + if (isTruncate) { if (log.isDebugEnabled()) { log.debug("[{}] Ledger {} will be truncated with ts {}", name, ls.getLedgerId(), ls.getTimestamp()); @@ -2542,70 +2561,107 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { advanceCursorsIfNecessary(ledgersToDelete); PositionImpl currentLastConfirmedEntry = lastConfirmedEntry; - // Update metadata - for (LedgerInfo ls : ledgersToDelete) { - if (currentLastConfirmedEntry != null && ls.getLedgerId() == currentLastConfirmedEntry.getLedgerId()) { - // this info is relevant because the lastMessageId won't be available anymore - log.info("[{}] Ledger {} contains the current last confirmed entry {}, and it is going to be " - + "deleted", name, ls.getLedgerId(), currentLastConfirmedEntry); - } - invalidateReadHandle(ls.getLedgerId()); - - ledgers.remove(ls.getLedgerId()); - NUMBER_OF_ENTRIES_UPDATER.addAndGet(this, -ls.getEntries()); - TOTAL_SIZE_UPDATER.addAndGet(this, -ls.getSize()); + // Update metadata + // Mark deletable ledgers + Set deletableLedgers = + Stream.concat(ledgersToDelete.stream().filter(ls -> !ls.getOffloadContext().getBookkeeperDeleted()), + offloadedLedgersToDelete.stream()).map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); + + // Mark deletable offloaded ledgers + Set deletableOffloadedLedgers = ledgersToDelete.stream() + .filter(ls -> ls.getOffloadContext().hasUidMsb() && ls.getOffloadContext().getComplete()) + .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); + + CompletableFuture updateTrashFuture = asyncUpdateTrashData(deletableLedgers, deletableOffloadedLedgers); + updateTrashFuture.thenAccept(ignore -> { + for (LedgerInfo ls : ledgersToDelete) { + if (currentLastConfirmedEntry != null && ls.getLedgerId() == currentLastConfirmedEntry.getLedgerId()) { + // this info is relevant because the lastMessageId won't be available anymore + log.info("[{}] Ledger {} contains the current last confirmed entry {}, and it is going to be " + + "deleted", name, ls.getLedgerId(), currentLastConfirmedEntry); + } - entryCache.invalidateAllEntries(ls.getLedgerId()); - } - for (LedgerInfo ls : offloadedLedgersToDelete) { - LedgerInfo.Builder newInfoBuilder = ls.toBuilder(); - newInfoBuilder.getOffloadContextBuilder().setBookkeeperDeleted(true); - String driverName = OffloadUtils.getOffloadDriverName(ls, - config.getLedgerOffloader().getOffloadDriverName()); - Map driverMetadata = OffloadUtils.getOffloadDriverMetadata(ls, - config.getLedgerOffloader().getOffloadDriverMetadata()); - OffloadUtils.setOffloadDriverMetadata(newInfoBuilder, driverName, driverMetadata); - ledgers.put(ls.getLedgerId(), newInfoBuilder.build()); - } + invalidateReadHandle(ls.getLedgerId()); + ledgers.remove(ls.getLedgerId()); + entryCache.invalidateAllEntries(ls.getLedgerId()); - if (log.isDebugEnabled()) { - log.debug("[{}] Updating of ledgers list after trimming", name); - } + NUMBER_OF_ENTRIES_UPDATER.addAndGet(this, -ls.getEntries()); + TOTAL_SIZE_UPDATER.addAndGet(this, -ls.getSize()); + } + for (LedgerInfo ls : offloadedLedgersToDelete) { + LedgerInfo.Builder newInfoBuilder = ls.toBuilder(); + newInfoBuilder.getOffloadContextBuilder().setBookkeeperDeleted(true); + String driverName = OffloadUtils.getOffloadDriverName(ls, + config.getLedgerOffloader().getOffloadDriverName()); + Map driverMetadata = OffloadUtils.getOffloadDriverMetadata(ls, + config.getLedgerOffloader().getOffloadDriverMetadata()); + OffloadUtils.setOffloadDriverMetadata(newInfoBuilder, driverName, driverMetadata); + ledgers.put(ls.getLedgerId(), newInfoBuilder.build()); + } - store.asyncUpdateLedgerIds(name, getManagedLedgerInfo(), ledgersStat, new MetaStoreCallback() { - @Override - public void operationComplete(Void result, Stat stat) { - log.info("[{}] End TrimConsumedLedgers. ledgers={} totalSize={}", name, ledgers.size(), - TOTAL_SIZE_UPDATER.get(ManagedLedgerImpl.this)); - ledgersStat = stat; - metadataMutex.unlock(); - trimmerMutex.unlock(); + if (log.isDebugEnabled()) { + log.debug("[{}] Updating of ledgers list after trimming", name); + } - for (LedgerInfo ls : ledgersToDelete) { - log.info("[{}] Removing ledger {} - size: {}", name, ls.getLedgerId(), ls.getSize()); - asyncDeleteLedger(ls.getLedgerId(), ls); - } - for (LedgerInfo ls : offloadedLedgersToDelete) { - log.info("[{}] Deleting offloaded ledger {} from bookkeeper - size: {}", name, ls.getLedgerId(), - ls.getSize()); - asyncDeleteLedgerFromBookKeeper(ls.getLedgerId()); + store.asyncUpdateLedgerIds(name, getManagedLedgerInfo(), ledgersStat, new MetaStoreCallback() { + @Override + public void operationComplete(Void result, Stat stat) { + log.info("[{}] End TrimConsumedLedgers. ledgers={} totalSize={}", name, ledgers.size(), + TOTAL_SIZE_UPDATER.get(ManagedLedgerImpl.this)); + ledgersStat = stat; + metadataMutex.unlock(); + trimmerMutex.unlock(); + executor.executeOrdered(name, safeRun(managedTrash::triggerDelete)); + promise.complete(null); } - promise.complete(null); - } - @Override - public void operationFailed(MetaStoreException e) { - log.warn("[{}] Failed to update the list of ledgers after trimming", name, e); - metadataMutex.unlock(); - trimmerMutex.unlock(); + @Override + public void operationFailed(MetaStoreException e) { + log.warn("[{}] Failed to update the list of ledgers after trimming", name, e); + metadataMutex.unlock(); + trimmerMutex.unlock(); - promise.completeExceptionally(e); - } + promise.completeExceptionally(e); + } + }); + }).exceptionally(ex -> { + metadataMutex.unlock(); + trimmerMutex.unlock(); + promise.completeExceptionally(ex); + return null; }); } } + private CompletableFuture asyncUpdateTrashData(Collection deletableLedgerIds, + Collection deletableOffloadedLedgerIds) { + List> futures = + new ArrayList<>(deletableLedgerIds.size() + deletableOffloadedLedgerIds.size()); + for (Long ledgerId : deletableLedgerIds) { + futures.add(managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.DELETABLE_LEDGER_SUFFIX)); + } + for (Long ledgerId : deletableOffloadedLedgerIds) { + futures.add(managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), + ManagedTrash.DELETABLE_OFFLOADED_LEDGER_SUFFIX)); + } + + CompletableFuture future = new CompletableFuture<>(); + futures.add(future); + managedTrash.asyncUpdateTrashData(Optional.of(new ManagedTrash.TrashMetaStoreCallback() { + @Override + public void operationComplete(Void result) { + future.complete(null); + } + + @Override + public void operationFailed(MetaStoreException e) { + future.completeExceptionally(e); + } + })); + return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])); + } + /** * Non-durable cursors have to be moved forward when data is trimmed since they are not retain that data. * This method also addresses a corner case for durable cursors in which the cursor is caught up, i.e. the mark @@ -2737,25 +2793,6 @@ public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) { } } - private void asyncDeleteLedgerFromBookKeeper(long ledgerId) { - asyncDeleteLedger(ledgerId, DEFAULT_LEDGER_DELETE_RETRIES); - } - - private void asyncDeleteLedger(long ledgerId, LedgerInfo info) { - if (!info.getOffloadContext().getBookkeeperDeleted()) { - // only delete if it hasn't been previously deleted for offload - asyncDeleteLedger(ledgerId, DEFAULT_LEDGER_DELETE_RETRIES); - } - - if (info.getOffloadContext().hasUidMsb()) { - UUID uuid = new UUID(info.getOffloadContext().getUidMsb(), info.getOffloadContext().getUidLsb()); - cleanupOffloaded(ledgerId, uuid, - OffloadUtils.getOffloadDriverName(info, config.getLedgerOffloader().getOffloadDriverName()), - OffloadUtils.getOffloadDriverMetadata(info, config.getLedgerOffloader().getOffloadDriverMetadata()), - "Trimming"); - } - } - private void asyncDeleteLedger(long ledgerId, long retry) { if (retry <= 0) { log.warn("[{}] Failed to delete ledger after retries {}", name, ledgerId); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java new file mode 100644 index 0000000000000..0c6442100ac60 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -0,0 +1,619 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import static org.apache.bookkeeper.mledger.util.Errors.isNoSuchLedgerExistsException; +import static org.apache.bookkeeper.mledger.util.SafeRun.safeRun; +import com.google.common.collect.Maps; +import com.google.protobuf.InvalidProtocolBufferException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedTrash; +import org.apache.bookkeeper.mledger.ManagedTrashMXBean; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.TrashDataComponent; +import org.apache.bookkeeper.mledger.util.CallbackMutex; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.Stat; +import org.apache.pulsar.metadata.api.extended.CreateOption; +import org.apache.pulsar.metadata.impl.batching.AbstractBatchedMetadataStore; +import org.apache.pulsar.metadata.impl.batching.MetadataOp; +import org.apache.pulsar.metadata.impl.batching.OpPut; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ManagedTrashImpl implements ManagedTrash { + + private static final Logger log = LoggerFactory.getLogger(ManagedTrashImpl.class); + + private static final String BASE_NODE = "/trash-data"; + + private static final String PREFIX = BASE_NODE + "/"; + + private static final String DELETE = "/delete"; + + private static final String ARCHIVE = "/archive-"; + + private static final String TRASH_KEY_SEPARATOR = "-"; + + private static final int RETRY_COUNT = 9; + + private static final CompletableFuture COMPLETED_FUTURE = CompletableFuture.completedFuture(null); + + //key:ledgerId value:storageContext + private NavigableMap trashData = new ConcurrentSkipListMap<>(); + + private final AtomicInteger toArchiveCount = new AtomicInteger(); + + private final CallbackMutex trashMutex = new CallbackMutex(); + + private final CallbackMutex deleteMutex = new CallbackMutex(); + + private final CallbackMutex trashPersistMutex = new CallbackMutex(); + + private final AbstractBatchedMetadataStore metadataStore; + + private volatile Stat deleteStat; + + private final String type; + + private final String name; + + private final ManagedLedgerConfig config; + + private final OrderedScheduler scheduledExecutor; + + private final OrderedExecutor executor; + + private final BookKeeper bookKeeper; + + private final int trashDataLimitSize; + + private volatile boolean trashIsDirty; + + private ScheduledFuture checkTrashPersistTask; + + private ScheduledFuture triggerDeleteTask; + + private final ManagedTrashMXBean managedTrashMXBean; + + public ManagedTrashImpl(String type, String name, MetadataStore metadataStore, ManagedLedgerConfig config, + OrderedScheduler scheduledExecutor, OrderedExecutor executor, BookKeeper bookKeeper) { + this.type = type; + this.name = name; + this.config = config; + if (!(metadataStore instanceof AbstractBatchedMetadataStore)) { + throw new IllegalStateException("ManagedTrashImpl metadata store must support batch operation."); + } + this.metadataStore = (AbstractBatchedMetadataStore) metadataStore; + this.scheduledExecutor = scheduledExecutor; + this.executor = executor; + this.bookKeeper = bookKeeper; + this.trashDataLimitSize = config.getTrashDataLimitSize(); + this.managedTrashMXBean = new ManagedTrashMXBeanImpl(this); + } + + @Override + public String name() { + return name + "-" + type; + } + + @Override + public CompletableFuture initialize() { + CompletableFuture future = new CompletableFuture<>(); + metadataStore.get(buildDeletePath()).whenCompleteAsync((res, e) -> { + if (e != null) { + log.error("Get delete data failed, name:{} type: {}", name, type, e); + future.completeExceptionally(e); + } else { + if (res.isEmpty()) { + future.complete(null); + return; + } + byte[] value = res.get().getValue(); + try { + trashData.putAll(deSerialize(value)); + deleteStat = res.get().getStat(); + toArchiveCount.set(calculateArchiveCount().get()); + future.complete(null); + checkTrashPersistTask = + scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, + TimeUnit.MINUTES); + triggerDeleteTask = + scheduledExecutor.scheduleAtFixedRate(safeRun(this::triggerDeleteInBackground), 1L, 1L, + TimeUnit.MINUTES); + } catch (InvalidProtocolBufferException exc) { + future.completeExceptionally(getException(exc)); + } catch (ExecutionException | InterruptedException ex) { + ex.printStackTrace(); + } + } + }, scheduledExecutor.chooseThread(name)); + return future; + } + + private void persistTrashIfNecessary() { + if (trashIsDirty) { + asyncUpdateTrashData(Optional.empty()); + } + } + + private CompletableFuture calculateArchiveCount() { + CompletableFuture future = new CompletableFuture<>(); + internalCalculateArchiveCount(future); + return future; + } + + private void internalCalculateArchiveCount(CompletableFuture future) { + if (!trashMutex.tryLock()) { + scheduledExecutor.schedule(safeRun(() -> internalCalculateArchiveCount(future)), 100, + TimeUnit.MILLISECONDS); + return; + } + try { + int toArchiveCount = 0; + for (Map.Entry entry : trashData.entrySet()) { + if (!entry.getKey().startsWith("0")) { + break; + } + toArchiveCount++; + } + future.complete(toArchiveCount); + } finally { + trashMutex.unlock(); + } + } + + @Override + public CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) { + CompletableFuture future = new CompletableFuture<>(); + appendTrashData(buildKey(RETRY_COUNT, ledgerId, type), context, future); + return future; + } + + private static ManagedLedgerException.MetaStoreException getException(Throwable t) { + if (t.getCause() instanceof MetadataStoreException.BadVersionException) { + return new ManagedLedgerException.BadVersionException(t.getMessage()); + } else { + return new ManagedLedgerException.MetaStoreException(t); + } + } + + public void appendInBackground(final String key, final LedgerInfo context, final CompletableFuture future) { + executor.executeOrdered(name, safeRun(() -> appendTrashData(key, context, future))); + } + + private void appendTrashData(final String key, final LedgerInfo context, final CompletableFuture future) { + if (!trashMutex.tryLock()) { + scheduledExecutor.schedule(safeRun(() -> appendInBackground(key, context, future)), 100, + TimeUnit.MILLISECONDS); + return; + } + try { + trashData.put(key, context); + trashIsDirty = true; + } finally { + trashMutex.unlock(); + } + } + + @Override + public void asyncUpdateTrashData(Optional> callback) { + metadataStore.put(buildDeletePath(), serialize(trashData), + deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion())) + .whenCompleteAsync((res, e) -> { + if (e != null) { + callback.ifPresent(call -> call.operationFailed(getException(e))); + return; + } + deleteStat = res; + trashIsDirty = false; + trashPersistMutex.unlock(); + callback.ifPresent(call -> call.operationComplete(null)); + }, executor.chooseThread(name)); + } + + private byte[] serialize(Map toPersist) { + TrashDataComponent.Builder builder = TrashDataComponent.newBuilder(); + for (Map.Entry entry : toPersist.entrySet()) { + builder.addComponent( + MLDataFormats.TrashData.newBuilder().setKey(entry.getKey()).setValue(entry.getValue()).build()); + } + return builder.build().toByteArray(); + } + + private Map deSerialize(byte[] content) throws InvalidProtocolBufferException { + TrashDataComponent component = TrashDataComponent.parseFrom(content); + List componentList = component.getComponentList(); + Map result = new HashMap<>(); + for (MLDataFormats.TrashData ele : componentList) { + result.put(ele.getKey(), ele.getValue()); + } + return result; + } + + private void triggerDeleteInBackground() { + executor.executeOrdered(name, safeRun(this::triggerDelete)); + } + + @Override + public void triggerDelete() { + if (!deleteMutex.tryLock()) { + scheduledExecutor.schedule(this::triggerDeleteInBackground, 100, TimeUnit.MILLISECONDS); + return; + } + if (!trashMutex.tryLock()) { + deleteMutex.unlock(); + scheduledExecutor.schedule(this::triggerDeleteInBackground, 100, TimeUnit.MILLISECONDS); + return; + } + List toDelete = getToDeleteData(); + if (toDelete.size() == 0) { + deleteMutex.unlock(); + trashMutex.unlock(); + return; + } + for (TrashDeleteHelper delHelper : toDelete) { + //unlock in asyncDeleteTrash + asyncDeleteTrash(delHelper); + } + } + + @Override + public List getAllArchiveIndex() { + return null; + } + + @Override + public Map getArchiveData(final long index) { + return null; + } + + @Override + public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { + if (checkTrashPersistTask != null) { + checkTrashPersistTask.cancel(true); + } + if (triggerDeleteTask != null) { + triggerDeleteTask.cancel(true); + } + asyncUpdateArchiveData(new TrashMetaStoreCallback<>() { + @Override + public void operationComplete(Void result) { + callback.closeComplete(ctx); + } + + @Override + public void operationFailed(ManagedLedgerException.MetaStoreException e) { + callback.closeFailed(getException(e), ctx); + } + }); + } + + private void increaseArchiveCountWhenDeleteFailed(final CompletableFuture future) { + toArchiveCount.incrementAndGet(); + updateArchiveDataIfNecessary(future); + } + + private void updateArchiveDataIfNecessary(final CompletableFuture future) { + if (toArchiveCount.get() < trashDataLimitSize / 2) { + future.complete(null); + return; + } + asyncUpdateArchiveData(new TrashMetaStoreCallback<>() { + @Override + public void operationComplete(Void result) { + future.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException.MetaStoreException e) { + future.completeExceptionally(e); + } + }); + } + + + private String buildPath() { + return PREFIX + name + "/" + type; + } + + private String buildDeletePath() { + return buildPath() + DELETE; + } + + private String buildArchivePath(long ledgerId) { + return buildPath() + ARCHIVE + ledgerId; + } + + //take 1/10 trash to delete, if the size over 10, use 10 to delete. + private List getToDeleteData() { + if (trashData.size() == 0) { + return Collections.emptyList(); + } + int batchSize = trashData.size() / 10; + if (batchSize > 10) { + batchSize = 10; + } + if (batchSize == 0) { + batchSize = 1; + } + List toDelete = new ArrayList<>(batchSize); + for (Map.Entry entry : trashData.descendingMap().entrySet()) { + TrashDeleteHelper delHelper = TrashDeleteHelper.build(entry.getKey(), entry.getValue()); + //if last retryCount is zero, the before data retryCount is zero too. + if (delHelper.retryCount == 0) { + break; + } + toDelete.add(delHelper); + if (toDelete.size() == batchSize) { + break; + } + } + return toDelete; + } + + private void asyncUpdateArchiveData(ManagedTrash.TrashMetaStoreCallback callback) { + //transaction operation + NavigableMap persistDelete = new ConcurrentSkipListMap<>(); + NavigableMap persistArchive = new ConcurrentSkipListMap<>(); + + + for (Map.Entry entry : trashData.entrySet()) { + persistArchive.put(entry.getKey(), entry.getValue()); + if (persistArchive.size() >= trashDataLimitSize / 2) { + break; + } + } + + persistDelete.putAll(trashData); + for (Map.Entry entry : persistArchive.entrySet()) { + persistDelete.remove(entry.getKey()); + } + //build delete persist operation + List txOps = new ArrayList<>(2); + OpPut opDeletePersist = new OpPut(buildDeletePath(), serialize(persistDelete), + deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion()), + EnumSet.noneOf(CreateOption.class)); + //build archive persist operation + Map.Entry lastEntry = persistArchive.lastEntry(); + OpPut opArchivePersist = + new OpPut(buildArchivePath(TrashDeleteHelper.build(lastEntry.getKey(), lastEntry.getValue()).ledgerId), + serialize(persistArchive), Optional.of(-1L), EnumSet.noneOf(CreateOption.class)); + txOps.add(opDeletePersist); + txOps.add(opArchivePersist); + metadataStore.batchOperation(txOps); + + opDeletePersist.getFuture().whenCompleteAsync((res, e) -> { + if (e != null) { + log.error("Persist trash data failed.", e); + callback.operationFailed(getException(e)); + return; + } + opArchivePersist.getFuture().whenComplete((res1, e1) -> { + if (e1 != null) { + log.error("Persist archive data failed.", e1); + callback.operationFailed(getException(e1)); + return; + } + deleteStat = res; + trashData = persistDelete; + trashIsDirty = false; + toArchiveCount.set(0); + }); + }, executor.chooseThread(name)); + + } + + private static String buildKey(int retryCount, long ledgerId, String suffix) { + return retryCount + TRASH_KEY_SEPARATOR + String.format("%019d", ledgerId) + TRASH_KEY_SEPARATOR + + suffix; + } + + private void asyncDeleteTrash(TrashDeleteHelper delHelper) { + if (delHelper.isLedger()) { + asyncDeleteLedger(delHelper.ledgerId, new AsyncCallbacks.DeleteLedgerCallback() { + @Override + public void deleteLedgerComplete(Object ctx) { + onDeleteSuccess(delHelper); + } + + @Override + public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { + onDeleteFailed(delHelper); + } + }); + } else if (delHelper.isOffloadLedger()) { + asyncDeleteOffloadedLedger(delHelper.ledgerId, delHelper.context, + new AsyncCallbacks.DeleteLedgerCallback() { + @Override + public void deleteLedgerComplete(Object ctx) { + onDeleteSuccess(delHelper); + } + + @Override + public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { + onDeleteFailed(delHelper); + } + }); + } + } + + private void onDeleteSuccess(TrashDeleteHelper delHelper) { + try { + String key = delHelper.transferToTrashKey(); + if (log.isDebugEnabled()) { + String info = null; + if (delHelper.isLedger()) { + info = String.format("Delete ledger %s success.", delHelper.ledgerId); + } else if (delHelper.isOffloadLedger()) { + info = String.format("Delete offload ledger %s success.", delHelper.ledgerId); + } + log.debug(info); + } + trashData.remove(key); + trashIsDirty = true; + } finally { + deleteMutex.unlock(); + trashMutex.unlock(); + } + } + + private void onDeleteFailed(TrashDeleteHelper delHelper) { + try { + //override old key + String key = delHelper.transferToTrashKey(); + trashData.remove(key); + trashData.put(buildKey(delHelper.retryCount - 1, delHelper.ledgerId, delHelper.suffix), + delHelper.context); + trashIsDirty = true; + if (delHelper.retryCount - 1 == 0) { + if (log.isDebugEnabled()) { + String info = null; + if (delHelper.isLedger()) { + info = String.format("Delete ledger %d reach retry limit %d.", delHelper.ledgerId, RETRY_COUNT); + } else if (delHelper.isOffloadLedger()) { + info = String.format("Delete offload ledger %d reach retry limit %d.", delHelper.ledgerId, + RETRY_COUNT); + } + log.debug(info); + } + increaseArchiveCountWhenDeleteFailed(COMPLETED_FUTURE); + } + } finally { + deleteMutex.unlock(); + trashMutex.unlock(); + } + } + + private void asyncDeleteLedger(long ledgerId, AsyncCallbacks.DeleteLedgerCallback callback) { + log.info("[{}] Removing ledger {}", name, ledgerId); + bookKeeper.asyncDeleteLedger(ledgerId, (rc, ctx) -> { + if (isNoSuchLedgerExistsException(rc)) { + log.warn("[{}] Ledger was already deleted {}", name, ledgerId); + } else if (rc != BKException.Code.OK) { + log.error("[{}] Error delete ledger {} : {}", name, ledgerId, BKException.getMessage(rc)); + callback.deleteLedgerFailed(ManagedLedgerImpl.createManagedLedgerException(rc), null); + return; + } + if (log.isDebugEnabled()) { + log.debug("[{}] Deleted ledger {}", name, ledgerId); + } + callback.deleteLedgerComplete(ctx); + }, null); + } + + private void asyncDeleteOffloadedLedger(long ledgerId, LedgerInfo info, + AsyncCallbacks.DeleteLedgerCallback callback) { + if (!info.getOffloadContext().hasUidMsb()) { + return; + } + String cleanupReason = "Trash-Trimming"; + + UUID uuid = new UUID(info.getOffloadContext().getUidMsb(), info.getOffloadContext().getUidLsb()); + + log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.", name, ledgerId, uuid, + cleanupReason); + Map metadataMap = Maps.newHashMap(); + metadataMap.putAll(config.getLedgerOffloader().getOffloadDriverMetadata()); + metadataMap.put("ManagedLedgerName", name); + + try { + config.getLedgerOffloader() + .deleteOffloaded(ledgerId, uuid, metadataMap) + .whenComplete((ignored, exception) -> { + if (exception != null) { + log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})", + name, ledgerId, cleanupReason, exception); + log.warn("[{}] Failed to delete offloaded ledger after retries {} / {}", name, ledgerId, + uuid); + callback.deleteLedgerFailed( + new ManagedLedgerException("Failed to delete offloaded ledger after retries"), + null); + return; + } + callback.deleteLedgerComplete(null); + }); + } catch (Exception e) { + log.warn("[{}] Failed to cleanup offloaded ledgers.", name, e); + } + } + + + private static class TrashDeleteHelper { + + private final int retryCount; + + private final long ledgerId; + + private final String suffix; + + private final LedgerInfo context; + + public TrashDeleteHelper(int retryCount, long ledgerId, String suffix, LedgerInfo context) { + this.retryCount = retryCount; + this.ledgerId = ledgerId; + this.suffix = suffix; + this.context = context; + } + + public static TrashDeleteHelper build(String key, LedgerInfo context) { + String[] split = key.split(TRASH_KEY_SEPARATOR); + int retryCont = Integer.parseInt(split[0]); + long ledgerId = Long.parseLong(split[1]); + return new TrashDeleteHelper(retryCont, ledgerId, split[2], context); + } + + private String transferToTrashKey() { + return ManagedTrashImpl.buildKey(retryCount, ledgerId, suffix); + } + + private boolean isLedger() { + return DELETABLE_LEDGER_SUFFIX.equals(suffix); + } + + private boolean isOffloadLedger() { + return DELETABLE_OFFLOADED_LEDGER_SUFFIX.equals(suffix); + } + } + + +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java new file mode 100644 index 0000000000000..4949749417000 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import org.apache.bookkeeper.mledger.ManagedTrash; +import org.apache.bookkeeper.mledger.ManagedTrashMXBean; + +public class ManagedTrashMXBeanImpl implements ManagedTrashMXBean { + + private final ManagedTrash managedTrash; + + public ManagedTrashMXBeanImpl(ManagedTrash managedTrash) { + this.managedTrash = managedTrash; + } + + @Override + public String getName() { + return managedTrash.name(); + } + + @Override + public String getType() { + return null; + } + + @Override + public long getCurrentNumberOfLedgersWaitingToDeleted() { + return managedTrash.; + } + + @Override + public long getTotalNumberOfLedgersWaitingToDeleted() { + return ; + } + + @Override + public long getCurrentNumberOfLedgersWaitingToArchive() { + return 0; + } + + @Override + public long getTotalNumberOfLedgersWaitingToArchive() { + return 0; + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 214c3afe1bcdb..d3c4d6baa615d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -34,15 +34,16 @@ import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Stat; @Slf4j public class ReadOnlyManagedLedgerImpl extends ManagedLedgerImpl { public ReadOnlyManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, - ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - String name) { - super(factory, bookKeeper, store, config, scheduledExecutor, name); + MetadataStore metadataStore, ManagedLedgerConfig config, + OrderedScheduler scheduledExecutor, String name) { + super(factory, bookKeeper, store, metadataStore, config, scheduledExecutor, name); } CompletableFuture initializeAndCreateCursor(PositionImpl startPosition) { diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index 4671816c1a199..b0a025b95472c 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -142,3 +142,13 @@ message ManagedCursorInfoMetadata { required CompressionType compressionType = 1; required int32 uncompressedSize = 2; } + +message TrashDataComponent { + repeated TrashData component = 1; +} + +message TrashData { + required string key = 1; + optional ManagedLedgerInfo.LedgerInfo value = 2; +} + diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java index e1c26bead096a..707410cb2ccae 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java @@ -159,7 +159,7 @@ protected CompletableFuture storePut(String path, byte[] data, Optional ops) { + public void batchOperation(List ops) { try { Txn txn = kv.txn(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 6697934d56b3a..badb45f189d3f 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -149,7 +149,7 @@ protected void receivedSessionEvent(SessionEvent event) { } @Override - protected void batchOperation(List ops) { + public void batchOperation(List ops) { try { zkc.multi(ops.stream().map(this::convertOp).collect(Collectors.toList()), (rc, path, ctx, results) -> { if (results == null) { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java index 616cac289efea..dd17cd8f5512b 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java @@ -158,5 +158,5 @@ private void enqueue(MessagePassingQueue queue, MetadataOp op) { } } - protected abstract void batchOperation(List ops); + public abstract void batchOperation(List ops); } From 4731f73256ef669f5093d84d1e4f6a2ffd2799aa Mon Sep 17 00:00:00 2001 From: horizonzy Date: Sat, 28 May 2022 17:07:28 +0800 Subject: [PATCH 03/43] fix compile problem. --- .../apache/bookkeeper/mledger/ManagedTrashMXBean.java | 2 -- .../bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java | 9 ++------- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java index 34ac52b99f2bd..2b0e9695b429e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java @@ -34,8 +34,6 @@ public interface ManagedTrashMXBean { long getTotalNumberOfLedgersWaitingToDeleted(); - void - long getCurrentNumberOfLedgersWaitingToArchive(); long getTotalNumberOfLedgersWaitingToArchive(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java index 4949749417000..8596a494b8355 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java @@ -34,19 +34,14 @@ public String getName() { return managedTrash.name(); } - @Override - public String getType() { - return null; - } - @Override public long getCurrentNumberOfLedgersWaitingToDeleted() { - return managedTrash.; + return 0; } @Override public long getTotalNumberOfLedgersWaitingToDeleted() { - return ; + return 0; } @Override From e37e5c46a85e9fd2f5ce111a9bbdd7e050a647e2 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Sat, 28 May 2022 17:12:26 +0800 Subject: [PATCH 04/43] fix code style. --- .../java/org/apache/bookkeeper/mledger/ManagedTrash.java | 6 +++--- .../apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 5 +++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 64e38f93e95d1..246246fac2865 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -47,7 +47,7 @@ interface TrashMetaStoreCallback { CompletableFuture initialize(); - CompletableFuture appendLedgerTrashData(final long ledgerId, final LedgerInfo context, final String type); + CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type); void asyncUpdateTrashData(Optional> callback); @@ -55,7 +55,7 @@ interface TrashMetaStoreCallback { List getAllArchiveIndex(); - Map getArchiveData(final long index); + Map getArchiveData(long index); - void asyncClose(final AsyncCallbacks.CloseCallback callback, final Object ctx); + void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx); } 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 67ff37a789fc3..5d610ff6d546c 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 @@ -340,8 +340,9 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper this.managedLedgerInterceptor = config.getManagedLedgerInterceptor(); } this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); - this.managedTrash = new ManagedTrashImpl(ManagedTrash.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, - executor, bookKeeper); + this.managedTrash = + new ManagedTrashImpl(ManagedTrash.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, + executor, bookKeeper); } synchronized void initialize(final ManagedLedgerInitializeLedgerCallback callback, final Object ctx) { From 5d04e3d55ae13cb0901882f582e77f6999737d64 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Sat, 28 May 2022 17:39:37 +0800 Subject: [PATCH 05/43] fix code format. --- .../mledger/impl/ManagedLedgerImpl.java | 26 ++++++++++++------- .../mledger/impl/ManagedTrashImpl.java | 2 +- 2 files changed, 17 insertions(+), 11 deletions(-) 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 5d610ff6d546c..6a992a9f98ca0 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 @@ -361,7 +361,8 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { if (mlInfo.hasTerminatedPosition()) { state = State.Terminated; lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); - log.info("[{}] Recovering managed ledger terminated at {}", name, lastConfirmedEntry); + log.info("[{}] Recovering managed ledger terminated at {}", name, + lastConfirmedEntry); } for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { @@ -386,19 +387,21 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { executor.executeOrdered(name, safeRun(() -> { mbean.endDataLedgerOpenOp(); if (log.isDebugEnabled()) { - log.debug("[{}] Opened ledger {}: {}", name, id, BKException.getMessage(rc)); + log.debug("[{}] Opened ledger {}: {}", name, id, + BKException.getMessage(rc)); } if (rc == BKException.Code.OK) { LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(id) - .setEntries(lh.getLastAddConfirmed() + 1).setSize(lh.getLength()) - .setTimestamp(clock.millis()).build(); + .setEntries(lh.getLastAddConfirmed() + 1) + .setSize(lh.getLength()).setTimestamp(clock.millis()).build(); ledgers.put(id, info); if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, lh) - .thenRun(() -> initializeBookKeeper(callback)) + managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, + lh).thenRun(() -> initializeBookKeeper(callback)) .exceptionally(ex -> { callback.initializeFailed( - new ManagedLedgerInterceptException(ex.getCause())); + new ManagedLedgerInterceptException( + ex.getCause())); return null; }); } else { @@ -409,7 +412,8 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { ledgers.remove(id); initializeBookKeeper(callback); } else { - log.error("[{}] Failed to open ledger {}: {}", name, id, BKException.getMessage(rc)); + log.error("[{}] Failed to open ledger {}: {}", name, id, + BKException.getMessage(rc)); callback.initializeFailed(createManagedLedgerException(rc)); return; } @@ -2574,10 +2578,12 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { .filter(ls -> ls.getOffloadContext().hasUidMsb() && ls.getOffloadContext().getComplete()) .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); - CompletableFuture updateTrashFuture = asyncUpdateTrashData(deletableLedgers, deletableOffloadedLedgers); + CompletableFuture updateTrashFuture = + asyncUpdateTrashData(deletableLedgers, deletableOffloadedLedgers); updateTrashFuture.thenAccept(ignore -> { for (LedgerInfo ls : ledgersToDelete) { - if (currentLastConfirmedEntry != null && ls.getLedgerId() == currentLastConfirmedEntry.getLedgerId()) { + if (currentLastConfirmedEntry != null + && ls.getLedgerId() == currentLastConfirmedEntry.getLedgerId()) { // this info is relevant because the lastMessageId won't be available anymore log.info("[{}] Ledger {} contains the current last confirmed entry {}, and it is going to be " + "deleted", name, ls.getLedgerId(), currentLastConfirmedEntry); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 0c6442100ac60..0589a01c84ee7 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, * software distributed under the License is distributed on an From b009d7e8e7bcc99d62cf0ecc39c8392ccf8094f3 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Sun, 29 May 2022 01:05:11 +0800 Subject: [PATCH 06/43] complete ManagedTrashMXBean --- .../bookkeeper/mledger/ManagedTrash.java | 4 +++ .../mledger/ManagedTrashMXBean.java | 10 +++++-- .../mledger/impl/ManagedTrashImpl.java | 12 ++++++++ .../mledger/impl/ManagedTrashMXBeanImpl.java | 29 ++++++++++++++----- 4 files changed, 45 insertions(+), 10 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 246246fac2865..80760cc1ccb36 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -57,5 +57,9 @@ interface TrashMetaStoreCallback { Map getArchiveData(long index); + long getTrashDataSize(); + + long getToArchiveDataSize(); + void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java index 2b0e9695b429e..612ed0e663d9e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrashMXBean.java @@ -30,13 +30,17 @@ public interface ManagedTrashMXBean { String getName(); - long getCurrentNumberOfLedgersWaitingToDeleted(); + long getCurrentNumberOfLedgersWaitingToDelete(); - long getTotalNumberOfLedgersWaitingToDeleted(); + void increaseTotalNumberOfDeleteLedgers(); + + long getTotalNumberOfDeleteLedgers(); long getCurrentNumberOfLedgersWaitingToArchive(); - long getTotalNumberOfLedgersWaitingToArchive(); + void increaseTotalNumberOfArchiveLedgers(); + + long getTotalNumberOfArchiveLedgers(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 0589a01c84ee7..50050f718ae87 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -229,6 +229,7 @@ private void appendTrashData(final String key, final LedgerInfo context, final C } try { trashData.put(key, context); + managedTrashMXBean.increaseTotalNumberOfDeleteLedgers();; trashIsDirty = true; } finally { trashMutex.unlock(); @@ -307,6 +308,16 @@ public Map getArchiveData(final long index) { return null; } + @Override + public long getTrashDataSize() { + return trashData.size(); + } + + @Override + public long getToArchiveDataSize() { + return toArchiveCount.get(); + } + @Override public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { if (checkTrashPersistTask != null) { @@ -330,6 +341,7 @@ public void operationFailed(ManagedLedgerException.MetaStoreException e) { private void increaseArchiveCountWhenDeleteFailed(final CompletableFuture future) { toArchiveCount.incrementAndGet(); + managedTrashMXBean.increaseTotalNumberOfArchiveLedgers(); updateArchiveDataIfNecessary(future); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java index 8596a494b8355..625876252af39 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.impl; +import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.mledger.ManagedTrash; import org.apache.bookkeeper.mledger.ManagedTrashMXBean; @@ -25,6 +26,10 @@ public class ManagedTrashMXBeanImpl implements ManagedTrashMXBean { private final ManagedTrash managedTrash; + private final AtomicLong totalNumberOfDeletedCount = new AtomicLong(); + + private final AtomicLong totalNumberOfArchiveCount = new AtomicLong(); + public ManagedTrashMXBeanImpl(ManagedTrash managedTrash) { this.managedTrash = managedTrash; } @@ -35,22 +40,32 @@ public String getName() { } @Override - public long getCurrentNumberOfLedgersWaitingToDeleted() { - return 0; + public long getCurrentNumberOfLedgersWaitingToDelete() { + return managedTrash.getTrashDataSize(); + } + + @Override + public void increaseTotalNumberOfDeleteLedgers() { + totalNumberOfDeletedCount.incrementAndGet(); } @Override - public long getTotalNumberOfLedgersWaitingToDeleted() { - return 0; + public long getTotalNumberOfDeleteLedgers() { + return totalNumberOfDeletedCount.get(); } @Override public long getCurrentNumberOfLedgersWaitingToArchive() { - return 0; + return managedTrash.getToArchiveDataSize(); + } + + @Override + public void increaseTotalNumberOfArchiveLedgers() { + totalNumberOfArchiveCount.incrementAndGet(); } @Override - public long getTotalNumberOfLedgersWaitingToArchive() { - return 0; + public long getTotalNumberOfArchiveLedgers() { + return totalNumberOfArchiveCount.get(); } } From 93ca968784753c1440662e87c3c35960039c7ebb Mon Sep 17 00:00:00 2001 From: horizonzy Date: Sun, 29 May 2022 01:28:13 +0800 Subject: [PATCH 07/43] fix trashData value null problem. --- .../org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 50050f718ae87..cbf1d7716615e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -78,6 +78,8 @@ public class ManagedTrashImpl implements ManagedTrash { private static final CompletableFuture COMPLETED_FUTURE = CompletableFuture.completedFuture(null); + private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().build(); + //key:ledgerId value:storageContext private NavigableMap trashData = new ConcurrentSkipListMap<>(); @@ -205,6 +207,9 @@ private void internalCalculateArchiveCount(CompletableFuture future) { @Override public CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) { CompletableFuture future = new CompletableFuture<>(); + if (context == null) { + context = EMPTY_LEDGER_INFO; + } appendTrashData(buildKey(RETRY_COUNT, ledgerId, type), context, future); return future; } From a13300722eb6c6274e741a42dfdf4302a8bf5571 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Sun, 29 May 2022 20:58:46 +0800 Subject: [PATCH 08/43] fix EMPTY_LEDGER_INFO initialize problem. --- .../org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index cbf1d7716615e..d12efc9925dc6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -78,7 +78,7 @@ public class ManagedTrashImpl implements ManagedTrash { private static final CompletableFuture COMPLETED_FUTURE = CompletableFuture.completedFuture(null); - private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().build(); + private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().setLedgerId(-1L).build(); //key:ledgerId value:storageContext private NavigableMap trashData = new ConcurrentSkipListMap<>(); From c5ca9799f4f946769c3760a0eee10018426fcf96 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 30 May 2022 12:52:48 +0800 Subject: [PATCH 09/43] remove triggerDelete schedule task. --- .../mledger/ManagedLedgerConfig.java | 10 +- .../bookkeeper/mledger/ManagedTrash.java | 10 +- .../mledger/impl/ManagedLedgerImpl.java | 12 +- .../mledger/impl/ManagedTrashImpl.java | 160 +++++++++++------- 4 files changed, 103 insertions(+), 89 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 83723ae9fcf24..ba884b89ee659 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -81,7 +81,7 @@ public class ManagedLedgerConfig { @Getter @Setter private boolean cacheEvictionByMarkDeletedPosition = false; - private int trashDataLimitSize = 1000; + private int archiveDataLimitSize = 500; public boolean isCreateIfMissing() { return createIfMissing; @@ -684,11 +684,11 @@ public void setInactiveLedgerRollOverTime(int inactiveLedgerRollOverTimeMs, Time this.inactiveLedgerRollOverTimeMs = (int) unit.toMillis(inactiveLedgerRollOverTimeMs); } - public int getTrashDataLimitSize() { - return trashDataLimitSize; + public int getArchiveDataLimitSize() { + return archiveDataLimitSize; } - public void setTrashDataLimitSize(int trashDataLimitSize) { - this.trashDataLimitSize = trashDataLimitSize; + public void setArchiveDataLimitSize(int archiveDataLimitSize) { + this.archiveDataLimitSize = archiveDataLimitSize; } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 80760cc1ccb36..32ba890c0cc7f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; @@ -36,20 +35,13 @@ public interface ManagedTrash { String DELETABLE_OFFLOADED_LEDGER_SUFFIX = "DOL"; - interface TrashMetaStoreCallback { - - void operationComplete(T result); - - void operationFailed(ManagedLedgerException.MetaStoreException e); - } - String name(); CompletableFuture initialize(); CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type); - void asyncUpdateTrashData(Optional> callback); + void asyncUpdateTrashData(CompletableFuture future); void triggerDelete(); 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 6a992a9f98ca0..0f7f279ac5244 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 @@ -2655,17 +2655,7 @@ private CompletableFuture asyncUpdateTrashData(Collection deletableL CompletableFuture future = new CompletableFuture<>(); futures.add(future); - managedTrash.asyncUpdateTrashData(Optional.of(new ManagedTrash.TrashMetaStoreCallback() { - @Override - public void operationComplete(Void result) { - future.complete(null); - } - - @Override - public void operationFailed(MetaStoreException e) { - future.completeExceptionally(e); - } - })); + managedTrash.asyncUpdateTrashData(future); return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index d12efc9925dc6..fc0c1c6f9722b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -37,6 +37,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.common.util.OrderedExecutor; @@ -80,6 +81,11 @@ public class ManagedTrashImpl implements ManagedTrash { private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().setLedgerId(-1L).build(); + private static final AtomicReferenceFieldUpdater STATE_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(ManagedTrashImpl.class, ManagedTrashImpl.State.class, "state"); + + protected volatile ManagedTrashImpl.State state = null; + //key:ledgerId value:storageContext private NavigableMap trashData = new ConcurrentSkipListMap<>(); @@ -107,14 +113,12 @@ public class ManagedTrashImpl implements ManagedTrash { private final BookKeeper bookKeeper; - private final int trashDataLimitSize; + private final int archiveDataLimitSize; private volatile boolean trashIsDirty; private ScheduledFuture checkTrashPersistTask; - private ScheduledFuture triggerDeleteTask; - private final ManagedTrashMXBean managedTrashMXBean; public ManagedTrashImpl(String type, String name, MetadataStore metadataStore, ManagedLedgerConfig config, @@ -125,11 +129,12 @@ public ManagedTrashImpl(String type, String name, MetadataStore metadataStore, M if (!(metadataStore instanceof AbstractBatchedMetadataStore)) { throw new IllegalStateException("ManagedTrashImpl metadata store must support batch operation."); } + STATE_UPDATER.set(this, State.None); this.metadataStore = (AbstractBatchedMetadataStore) metadataStore; this.scheduledExecutor = scheduledExecutor; this.executor = executor; this.bookKeeper = bookKeeper; - this.trashDataLimitSize = config.getTrashDataLimitSize(); + this.archiveDataLimitSize = config.getArchiveDataLimitSize(); this.managedTrashMXBean = new ManagedTrashMXBeanImpl(this); } @@ -145,36 +150,36 @@ public CompletableFuture initialize() { if (e != null) { log.error("Get delete data failed, name:{} type: {}", name, type, e); future.completeExceptionally(e); - } else { - if (res.isEmpty()) { - future.complete(null); - return; - } - byte[] value = res.get().getValue(); - try { - trashData.putAll(deSerialize(value)); - deleteStat = res.get().getStat(); - toArchiveCount.set(calculateArchiveCount().get()); - future.complete(null); - checkTrashPersistTask = - scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, - TimeUnit.MINUTES); - triggerDeleteTask = - scheduledExecutor.scheduleAtFixedRate(safeRun(this::triggerDeleteInBackground), 1L, 1L, - TimeUnit.MINUTES); - } catch (InvalidProtocolBufferException exc) { - future.completeExceptionally(getException(exc)); - } catch (ExecutionException | InterruptedException ex) { - ex.printStackTrace(); - } + return; + } + if (res.isEmpty()) { + future.complete(null); + return; + } + byte[] value = res.get().getValue(); + try { + trashData.putAll(deSerialize(value)); + deleteStat = res.get().getStat(); + toArchiveCount.set(calculateArchiveCount().get()); + future.complete(null); + checkTrashPersistTask = + scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, + TimeUnit.MINUTES); + STATE_UPDATER.set(this, State.INITIALIZED); + triggerDeleteInBackground(); + } catch (InvalidProtocolBufferException exc) { + future.completeExceptionally(getException(exc)); + } catch (ExecutionException | InterruptedException ex) { + ex.printStackTrace(); } + }, scheduledExecutor.chooseThread(name)); return future; } private void persistTrashIfNecessary() { if (trashIsDirty) { - asyncUpdateTrashData(Optional.empty()); + asyncUpdateTrashData(COMPLETED_FUTURE); } } @@ -227,6 +232,11 @@ public void appendInBackground(final String key, final LedgerInfo context, final } private void appendTrashData(final String key, final LedgerInfo context, final CompletableFuture future) { + if (STATE_UPDATER.get(this) != State.INITIALIZED) { + future.completeExceptionally( + getException(new IllegalStateException("ManagedTrashImpl is not initialized"))); + return; + } if (!trashMutex.tryLock()) { scheduledExecutor.schedule(safeRun(() -> appendInBackground(key, context, future)), 100, TimeUnit.MILLISECONDS); @@ -234,7 +244,7 @@ private void appendTrashData(final String key, final LedgerInfo context, final C } try { trashData.put(key, context); - managedTrashMXBean.increaseTotalNumberOfDeleteLedgers();; + managedTrashMXBean.increaseTotalNumberOfDeleteLedgers(); trashIsDirty = true; } finally { trashMutex.unlock(); @@ -242,18 +252,23 @@ private void appendTrashData(final String key, final LedgerInfo context, final C } @Override - public void asyncUpdateTrashData(Optional> callback) { + public void asyncUpdateTrashData(CompletableFuture future) { + if (STATE_UPDATER.get(this) != State.INITIALIZED) { + future.completeExceptionally( + getException(new IllegalStateException("ManagedTrashImpl is not initialized"))); + return; + } metadataStore.put(buildDeletePath(), serialize(trashData), deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion())) .whenCompleteAsync((res, e) -> { if (e != null) { - callback.ifPresent(call -> call.operationFailed(getException(e))); + future.completeExceptionally(getException(e)); return; } deleteStat = res; trashIsDirty = false; trashPersistMutex.unlock(); - callback.ifPresent(call -> call.operationComplete(null)); + future.complete(null); }, executor.chooseThread(name)); } @@ -282,13 +297,15 @@ private void triggerDeleteInBackground() { @Override public void triggerDelete() { + if (STATE_UPDATER.get(this) != State.INITIALIZED) { + return; + } if (!deleteMutex.tryLock()) { - scheduledExecutor.schedule(this::triggerDeleteInBackground, 100, TimeUnit.MILLISECONDS); return; } if (!trashMutex.tryLock()) { - deleteMutex.unlock(); scheduledExecutor.schedule(this::triggerDeleteInBackground, 100, TimeUnit.MILLISECONDS); + deleteMutex.unlock(); return; } List toDelete = getToDeleteData(); @@ -325,22 +342,18 @@ public long getToArchiveDataSize() { @Override public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { + STATE_UPDATER.set(this, State.Closed); if (checkTrashPersistTask != null) { checkTrashPersistTask.cancel(true); } - if (triggerDeleteTask != null) { - triggerDeleteTask.cancel(true); - } - asyncUpdateArchiveData(new TrashMetaStoreCallback<>() { - @Override - public void operationComplete(Void result) { - callback.closeComplete(ctx); - } - - @Override - public void operationFailed(ManagedLedgerException.MetaStoreException e) { - callback.closeFailed(getException(e), ctx); + CompletableFuture future = new CompletableFuture<>(); + asyncUpdateTrashData(future); + future.whenComplete((res, e) -> { + if (e != null) { + callback.closeFailed((ManagedLedgerException) e, ctx); + return; } + callback.closeComplete(ctx); }); } @@ -351,21 +364,11 @@ private void increaseArchiveCountWhenDeleteFailed(final CompletableFuture fut } private void updateArchiveDataIfNecessary(final CompletableFuture future) { - if (toArchiveCount.get() < trashDataLimitSize / 2) { + if (toArchiveCount.get() < archiveDataLimitSize) { future.complete(null); return; } - asyncUpdateArchiveData(new TrashMetaStoreCallback<>() { - @Override - public void operationComplete(Void result) { - future.complete(null); - } - - @Override - public void operationFailed(ManagedLedgerException.MetaStoreException e) { - future.completeExceptionally(e); - } - }); + asyncUpdateArchiveData(future); } @@ -408,7 +411,7 @@ private List getToDeleteData() { return toDelete; } - private void asyncUpdateArchiveData(ManagedTrash.TrashMetaStoreCallback callback) { + private void asyncUpdateArchiveData(CompletableFuture future) { //transaction operation NavigableMap persistDelete = new ConcurrentSkipListMap<>(); NavigableMap persistArchive = new ConcurrentSkipListMap<>(); @@ -416,7 +419,7 @@ private void asyncUpdateArchiveData(ManagedTrash.TrashMetaStoreCallback ca for (Map.Entry entry : trashData.entrySet()) { persistArchive.put(entry.getKey(), entry.getValue()); - if (persistArchive.size() >= trashDataLimitSize / 2) { + if (persistArchive.size() >= archiveDataLimitSize) { break; } } @@ -433,7 +436,7 @@ private void asyncUpdateArchiveData(ManagedTrash.TrashMetaStoreCallback ca //build archive persist operation Map.Entry lastEntry = persistArchive.lastEntry(); OpPut opArchivePersist = - new OpPut(buildArchivePath(TrashDeleteHelper.build(lastEntry.getKey(), lastEntry.getValue()).ledgerId), + new OpPut(buildArchivePath(TrashDeleteHelper.build(lastEntry.getKey()).ledgerId), serialize(persistArchive), Optional.of(-1L), EnumSet.noneOf(CreateOption.class)); txOps.add(opDeletePersist); txOps.add(opArchivePersist); @@ -442,20 +445,21 @@ private void asyncUpdateArchiveData(ManagedTrash.TrashMetaStoreCallback ca opDeletePersist.getFuture().whenCompleteAsync((res, e) -> { if (e != null) { log.error("Persist trash data failed.", e); - callback.operationFailed(getException(e)); + future.completeExceptionally(getException(e)); return; } - opArchivePersist.getFuture().whenComplete((res1, e1) -> { + opArchivePersist.getFuture().whenCompleteAsync((res1, e1) -> { if (e1 != null) { log.error("Persist archive data failed.", e1); - callback.operationFailed(getException(e1)); + future.completeExceptionally(getException(e1)); return; } deleteStat = res; trashData = persistDelete; trashIsDirty = false; toArchiveCount.set(0); - }); + future.complete(null); + }, executor.chooseThread(name)); }, executor.chooseThread(name)); } @@ -509,8 +513,12 @@ private void onDeleteSuccess(TrashDeleteHelper delHelper) { trashData.remove(key); trashIsDirty = true; } finally { + boolean continueToDelete = continueToDelete(); deleteMutex.unlock(); trashMutex.unlock(); + if (continueToDelete) { + triggerDeleteInBackground(); + } } } @@ -536,11 +544,20 @@ private void onDeleteFailed(TrashDeleteHelper delHelper) { increaseArchiveCountWhenDeleteFailed(COMPLETED_FUTURE); } } finally { + boolean continueToDelete = continueToDelete(); deleteMutex.unlock(); trashMutex.unlock(); + if (continueToDelete) { + triggerDeleteInBackground(); + } } } + private boolean continueToDelete() { + TrashDeleteHelper delHelper = TrashDeleteHelper.build(trashData.lastEntry().getKey()); + return delHelper.retryCount > 0; + } + private void asyncDeleteLedger(long ledgerId, AsyncCallbacks.DeleteLedgerCallback callback) { log.info("[{}] Removing ledger {}", name, ledgerId); bookKeeper.asyncDeleteLedger(ledgerId, (rc, ctx) -> { @@ -605,6 +622,7 @@ private static class TrashDeleteHelper { private final LedgerInfo context; + public TrashDeleteHelper(int retryCount, long ledgerId, String suffix, LedgerInfo context) { this.retryCount = retryCount; this.ledgerId = ledgerId; @@ -612,6 +630,14 @@ public TrashDeleteHelper(int retryCount, long ledgerId, String suffix, LedgerInf this.context = context; } + public static TrashDeleteHelper build(String key) { + String[] split = key.split(TRASH_KEY_SEPARATOR); + int retryCont = Integer.parseInt(split[0]); + long ledgerId = Long.parseLong(split[1]); + return new TrashDeleteHelper(retryCont, ledgerId, split[2], null); + } + + public static TrashDeleteHelper build(String key, LedgerInfo context) { String[] split = key.split(TRASH_KEY_SEPARATOR); int retryCont = Integer.parseInt(split[0]); @@ -632,5 +658,11 @@ private boolean isOffloadLedger() { } } + public enum State { + None, + INITIALIZED, + Closed, + } + } From 61a0e914e3269346464fd13fe8e44853e1fd7fcd Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 30 May 2022 14:10:48 +0800 Subject: [PATCH 10/43] code clean. --- .../bookkeeper/mledger/ManagedTrash.java | 2 +- .../mledger/impl/ManagedLedgerImpl.java | 5 +- .../mledger/impl/ManagedTrashImpl.java | 120 ++++++++++-------- 3 files changed, 68 insertions(+), 59 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 32ba890c0cc7f..278f489887d50 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -41,7 +41,7 @@ public interface ManagedTrash { CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type); - void asyncUpdateTrashData(CompletableFuture future); + CompletableFuture asyncUpdateTrashData(); void triggerDelete(); 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 0f7f279ac5244..bf4460f3a8e0f 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 @@ -2652,10 +2652,7 @@ private CompletableFuture asyncUpdateTrashData(Collection deletableL futures.add(managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), ManagedTrash.DELETABLE_OFFLOADED_LEDGER_SUFFIX)); } - - CompletableFuture future = new CompletableFuture<>(); - futures.add(future); - managedTrash.asyncUpdateTrashData(future); + futures.add(managedTrash.asyncUpdateTrashData()); return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index fc0c1c6f9722b..ff6bc41633355 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -33,7 +33,6 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -77,8 +76,6 @@ public class ManagedTrashImpl implements ManagedTrash { private static final int RETRY_COUNT = 9; - private static final CompletableFuture COMPLETED_FUTURE = CompletableFuture.completedFuture(null); - private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().setLedgerId(-1L).build(); private static final AtomicReferenceFieldUpdater STATE_UPDATER = @@ -148,7 +145,7 @@ public CompletableFuture initialize() { CompletableFuture future = new CompletableFuture<>(); metadataStore.get(buildDeletePath()).whenCompleteAsync((res, e) -> { if (e != null) { - log.error("Get delete data failed, name:{} type: {}", name, type, e); + log.error("[{}] Get delete data failed.", name(), e); future.completeExceptionally(e); return; } @@ -160,38 +157,45 @@ public CompletableFuture initialize() { try { trashData.putAll(deSerialize(value)); deleteStat = res.get().getStat(); - toArchiveCount.set(calculateArchiveCount().get()); - future.complete(null); - checkTrashPersistTask = - scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, - TimeUnit.MINUTES); - STATE_UPDATER.set(this, State.INITIALIZED); - triggerDeleteInBackground(); + calculateArchiveCount().whenComplete((res1, e1) -> { + if (e1 != null) { + future.completeExceptionally(getException(e1)); + return; + } + toArchiveCount.set(res1); + future.complete(null); + checkTrashPersistTask = + scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, + TimeUnit.MINUTES); + STATE_UPDATER.set(this, State.INITIALIZED); + triggerDeleteInBackground(); + }); } catch (InvalidProtocolBufferException exc) { future.completeExceptionally(getException(exc)); - } catch (ExecutionException | InterruptedException ex) { - ex.printStackTrace(); } - }, scheduledExecutor.chooseThread(name)); return future; } private void persistTrashIfNecessary() { if (trashIsDirty) { - asyncUpdateTrashData(COMPLETED_FUTURE); + asyncUpdateTrashData(); } } private CompletableFuture calculateArchiveCount() { CompletableFuture future = new CompletableFuture<>(); - internalCalculateArchiveCount(future); + calculateArchiveCountInBackground(future); return future; } + private void calculateArchiveCountInBackground(CompletableFuture future) { + executor.executeOrdered(name, safeRun(() -> internalCalculateArchiveCount(future))); + } + private void internalCalculateArchiveCount(CompletableFuture future) { if (!trashMutex.tryLock()) { - scheduledExecutor.schedule(safeRun(() -> internalCalculateArchiveCount(future)), 100, + scheduledExecutor.schedule(safeRun(() -> calculateArchiveCountInBackground(future)), 100, TimeUnit.MILLISECONDS); return; } @@ -232,9 +236,10 @@ public void appendInBackground(final String key, final LedgerInfo context, final } private void appendTrashData(final String key, final LedgerInfo context, final CompletableFuture future) { - if (STATE_UPDATER.get(this) != State.INITIALIZED) { - future.completeExceptionally( - getException(new IllegalStateException("ManagedTrashImpl is not initialized"))); + State state = STATE_UPDATER.get(this); + if (state != State.INITIALIZED) { + future.completeExceptionally(getException(new IllegalStateException( + String.format("[%s] is not initialized, current state: %s", name(), state)))); return; } if (!trashMutex.tryLock()) { @@ -252,11 +257,14 @@ private void appendTrashData(final String key, final LedgerInfo context, final C } @Override - public void asyncUpdateTrashData(CompletableFuture future) { - if (STATE_UPDATER.get(this) != State.INITIALIZED) { - future.completeExceptionally( - getException(new IllegalStateException("ManagedTrashImpl is not initialized"))); - return; + public CompletableFuture asyncUpdateTrashData() { + log.info("{} Start async update trash data", name()); + CompletableFuture future = new CompletableFuture<>(); + State state = STATE_UPDATER.get(this); + if (state != State.INITIALIZED) { + future.completeExceptionally(getException(new IllegalStateException( + String.format("[%s] is not initialized, current state: %s", name(), state)))); + return future; } metadataStore.put(buildDeletePath(), serialize(trashData), deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion())) @@ -270,6 +278,7 @@ public void asyncUpdateTrashData(CompletableFuture future) { trashPersistMutex.unlock(); future.complete(null); }, executor.chooseThread(name)); + return future; } private byte[] serialize(Map toPersist) { @@ -297,7 +306,9 @@ private void triggerDeleteInBackground() { @Override public void triggerDelete() { - if (STATE_UPDATER.get(this) != State.INITIALIZED) { + State state = STATE_UPDATER.get(this); + if (state != State.INITIALIZED) { + log.warn("[{}] is not initialized, current state: {}", name(), state); return; } if (!deleteMutex.tryLock()) { @@ -342,25 +353,26 @@ public long getToArchiveDataSize() { @Override public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { - STATE_UPDATER.set(this, State.Closed); if (checkTrashPersistTask != null) { checkTrashPersistTask.cancel(true); + checkTrashPersistTask = null; } - CompletableFuture future = new CompletableFuture<>(); - asyncUpdateTrashData(future); - future.whenComplete((res, e) -> { + asyncUpdateTrashData().whenComplete((res, e) -> { if (e != null) { callback.closeFailed((ManagedLedgerException) e, ctx); return; } callback.closeComplete(ctx); }); + STATE_UPDATER.set(this, State.Closed); } - private void increaseArchiveCountWhenDeleteFailed(final CompletableFuture future) { + private CompletableFuture increaseArchiveCountWhenDeleteFailed() { + CompletableFuture future = new CompletableFuture<>(); toArchiveCount.incrementAndGet(); managedTrashMXBean.increaseTotalNumberOfArchiveLedgers(); updateArchiveDataIfNecessary(future); + return future; } private void updateArchiveDataIfNecessary(final CompletableFuture future) { @@ -412,6 +424,7 @@ private List getToDeleteData() { } private void asyncUpdateArchiveData(CompletableFuture future) { + log.info("[{}] Start async update archive data", name()); //transaction operation NavigableMap persistDelete = new ConcurrentSkipListMap<>(); NavigableMap persistArchive = new ConcurrentSkipListMap<>(); @@ -444,13 +457,13 @@ private void asyncUpdateArchiveData(CompletableFuture future) { opDeletePersist.getFuture().whenCompleteAsync((res, e) -> { if (e != null) { - log.error("Persist trash data failed.", e); + log.error("[{}] Persist trash data failed.", name(), e); future.completeExceptionally(getException(e)); return; } opArchivePersist.getFuture().whenCompleteAsync((res1, e1) -> { if (e1 != null) { - log.error("Persist archive data failed.", e1); + log.error("[{}] Persist archive data failed.", name(), e1); future.completeExceptionally(getException(e1)); return; } @@ -504,9 +517,9 @@ private void onDeleteSuccess(TrashDeleteHelper delHelper) { if (log.isDebugEnabled()) { String info = null; if (delHelper.isLedger()) { - info = String.format("Delete ledger %s success.", delHelper.ledgerId); + info = String.format("[%s] Delete ledger %s success.", name(), delHelper.ledgerId); } else if (delHelper.isOffloadLedger()) { - info = String.format("Delete offload ledger %s success.", delHelper.ledgerId); + info = String.format("[%s] Delete offload ledger %s success.", name(), delHelper.ledgerId); } log.debug(info); } @@ -531,17 +544,18 @@ private void onDeleteFailed(TrashDeleteHelper delHelper) { delHelper.context); trashIsDirty = true; if (delHelper.retryCount - 1 == 0) { - if (log.isDebugEnabled()) { + if (log.isWarnEnabled()) { String info = null; if (delHelper.isLedger()) { - info = String.format("Delete ledger %d reach retry limit %d.", delHelper.ledgerId, RETRY_COUNT); - } else if (delHelper.isOffloadLedger()) { - info = String.format("Delete offload ledger %d reach retry limit %d.", delHelper.ledgerId, + info = String.format("[%s] Delete ledger %d reach retry limit %d.", name(), delHelper.ledgerId, RETRY_COUNT); + } else if (delHelper.isOffloadLedger()) { + info = String.format("[%s] Delete offload ledger %d reach retry limit %d.", name(), + delHelper.ledgerId, RETRY_COUNT); } - log.debug(info); + log.warn(info); } - increaseArchiveCountWhenDeleteFailed(COMPLETED_FUTURE); + increaseArchiveCountWhenDeleteFailed(); } } finally { boolean continueToDelete = continueToDelete(); @@ -559,17 +573,17 @@ private boolean continueToDelete() { } private void asyncDeleteLedger(long ledgerId, AsyncCallbacks.DeleteLedgerCallback callback) { - log.info("[{}] Removing ledger {}", name, ledgerId); + log.info("[{}] Start async delete ledger {}", name(), ledgerId); bookKeeper.asyncDeleteLedger(ledgerId, (rc, ctx) -> { if (isNoSuchLedgerExistsException(rc)) { - log.warn("[{}] Ledger was already deleted {}", name, ledgerId); + log.warn("[{}] Ledger was already deleted {}", name(), ledgerId); } else if (rc != BKException.Code.OK) { - log.error("[{}] Error delete ledger {} : {}", name, ledgerId, BKException.getMessage(rc)); + log.error("[{}] Error delete ledger {} : {}", name(), ledgerId, BKException.getMessage(rc)); callback.deleteLedgerFailed(ManagedLedgerImpl.createManagedLedgerException(rc), null); return; } if (log.isDebugEnabled()) { - log.debug("[{}] Deleted ledger {}", name, ledgerId); + log.debug("[{}] Deleted ledger {}", name(), ledgerId); } callback.deleteLedgerComplete(ctx); }, null); @@ -584,8 +598,9 @@ private void asyncDeleteOffloadedLedger(long ledgerId, LedgerInfo info, UUID uuid = new UUID(info.getOffloadContext().getUidMsb(), info.getOffloadContext().getUidLsb()); - log.info("[{}] Cleanup offload for ledgerId {} uuid {} because of the reason {}.", name, ledgerId, uuid, - cleanupReason); + log.info("[{}] Start async delete offloaded ledger, ledgerId {} uuid {} because of the reason {}.", name(), + ledgerId, uuid, cleanupReason); + Map metadataMap = Maps.newHashMap(); metadataMap.putAll(config.getLedgerOffloader().getOffloadDriverMetadata()); metadataMap.put("ManagedLedgerName", name); @@ -595,19 +610,16 @@ private void asyncDeleteOffloadedLedger(long ledgerId, LedgerInfo info, .deleteOffloaded(ledgerId, uuid, metadataMap) .whenComplete((ignored, exception) -> { if (exception != null) { - log.warn("[{}] Error cleaning up offload for {}, (cleanup reason: {})", - name, ledgerId, cleanupReason, exception); - log.warn("[{}] Failed to delete offloaded ledger after retries {} / {}", name, ledgerId, - uuid); - callback.deleteLedgerFailed( - new ManagedLedgerException("Failed to delete offloaded ledger after retries"), + log.warn("[{}] Failed delete offload for ledgerId {} uuid {}, (cleanup reason: {})", + name(), ledgerId, uuid, cleanupReason, exception); + callback.deleteLedgerFailed(new ManagedLedgerException("Failed to delete offloaded ledger"), null); return; } callback.deleteLedgerComplete(null); }); } catch (Exception e) { - log.warn("[{}] Failed to cleanup offloaded ledgers.", name, e); + log.warn("[{}] Failed to delete offloaded ledgers.", name(), e); } } From 464a407234dc52a28917312776493200fca568cd Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 30 May 2022 14:11:20 +0800 Subject: [PATCH 11/43] revert broker.conf --- conf/broker.conf | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 1f9b3389a238b..ee1eb3a7a6323 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -109,7 +109,7 @@ enableBusyWait=false isRunningStandalone= # Name of the cluster to which this broker belongs to -clusterName=horizon-cluster +clusterName= # The maximum number of tenants that each pulsar cluster can create # This configuration is not precise control, in a concurrent scenario, the threshold will be exceeded @@ -119,7 +119,7 @@ maxTenants=0 failureDomainsEnabled=false # Metadata store session timeout in milliseconds -metadataStoreSessionTimeoutMillis=600000 +metadataStoreSessionTimeoutMillis=30000 # Metadata store operation timeout in seconds metadataStoreOperationTimeoutSeconds=30 @@ -232,7 +232,7 @@ subscriptionKeySharedConsistentHashingReplicaPoints=100 # Set the default behavior for message deduplication in the broker # This can be overridden per-namespace. If enabled, broker will reject # messages that were already stored in the topic -brokerDeduplicationEnabled=true +brokerDeduplicationEnabled=false # Maximum number of producer information that it's going to be # persisted for deduplication purposes @@ -515,7 +515,7 @@ delayedDeliveryEnabled=true delayedDeliveryTickTimeMillis=1000 # Whether to enable acknowledge of batch local index. -acknowledgmentAtBatchIndexLevelEnabled=true +acknowledgmentAtBatchIndexLevelEnabled=false # Enable tracking of replicated subscriptions state across clusters. enableReplicatedSubscriptions=true @@ -810,7 +810,7 @@ maxConcurrentHttpRequests=1024 # For example: zk+hierarchical://localhost:2181/ledgers # The metadata service uri list can also be semicolon separated values like below: # zk+hierarchical://zk1:2181;zk2:2181;zk3:2181/ledgers -bookkeeperMetadataServiceUri=zk+hierarchical://127.0.0.1:2181/ledgers +bookkeeperMetadataServiceUri= # Authentication plugin to use when connecting to bookies bookkeeperClientAuthenticationPlugin= @@ -941,10 +941,10 @@ bookkeeperExplicitLacIntervalInMills=0 managedLedgerDefaultEnsembleSize=2 # Number of copies to store for each message -managedLedgerDefaultWriteQuorum=1 +managedLedgerDefaultWriteQuorum=2 # Number of guaranteed copies (acks to wait before write is complete) -managedLedgerDefaultAckQuorum=1 +managedLedgerDefaultAckQuorum=2 # with OpportunisticStriping=true the ensembleSize is adapted automatically to writeQuorum # in case of lack of enough bookies @@ -1440,12 +1440,12 @@ strictBookieAffinityEnabled=false # Zookeeper quorum connection string # Deprecated: use metadataStoreUrl instead -zookeeperServers=127.0.0.1:2181 +zookeeperServers= # Configuration Store connection string # Deprecated. Use configurationMetadataStoreUrl globalZookeeperServers= -configurationStoreServers=127.0.0.1:2181 +configurationStoreServers= # Zookeeper session timeout in milliseconds # Deprecated: use metadataStoreSessionTimeoutMillis From 3e571e25109c140abf923a010b9c564f606d6203 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 30 May 2022 15:02:16 +0800 Subject: [PATCH 12/43] optimize persist data when the context is empty. --- .../mledger/impl/ManagedTrashImpl.java | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 668bbcb9f8bc6..4005a88c6f054 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -76,7 +76,9 @@ public class ManagedTrashImpl implements ManagedTrash { private static final int RETRY_COUNT = 9; - private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().setLedgerId(-1L).build(); + private static final long EMPTY_LEDGER_ID = -1L; + + private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().setLedgerId(EMPTY_LEDGER_ID).build(); private static final AtomicReferenceFieldUpdater STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(ManagedTrashImpl.class, ManagedTrashImpl.State.class, "state"); @@ -284,8 +286,11 @@ public CompletableFuture asyncUpdateTrashData() { private byte[] serialize(Map toPersist) { TrashDataComponent.Builder builder = TrashDataComponent.newBuilder(); for (Map.Entry entry : toPersist.entrySet()) { - builder.addComponent( - MLDataFormats.TrashData.newBuilder().setKey(entry.getKey()).setValue(entry.getValue()).build()); + MLDataFormats.TrashData.Builder innerBuilder = MLDataFormats.TrashData.newBuilder().setKey(entry.getKey()); + if (entry.getValue().getLedgerId() != EMPTY_LEDGER_ID) { + innerBuilder.setValue(entry.getValue()); + } + builder.addComponent(innerBuilder.build()); } return builder.build().toByteArray(); } @@ -295,7 +300,11 @@ private Map deSerialize(byte[] content) throws InvalidProtoc List componentList = component.getComponentList(); Map result = new HashMap<>(); for (MLDataFormats.TrashData ele : componentList) { - result.put(ele.getKey(), ele.getValue()); + if (ele.hasValue()) { + result.put(ele.getKey(), ele.getValue()); + } else { + result.put(ele.getKey(), EMPTY_LEDGER_INFO); + } } return result; } From 699d05a1fe72bb7925e3bd55ac8003e30c1aa3a2 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Tue, 31 May 2022 00:44:15 +0800 Subject: [PATCH 13/43] complete admin api. --- .../bookkeeper/mledger/ManagedTrash.java | 4 +- .../mledger/impl/ManagedTrashImpl.java | 48 +++++++++++++++---- 2 files changed, 40 insertions(+), 12 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 278f489887d50..ebaeba5cccf46 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -45,9 +45,9 @@ public interface ManagedTrash { void triggerDelete(); - List getAllArchiveIndex(); + CompletableFuture> getAllArchiveIndex(); - Map getArchiveData(long index); + CompletableFuture> getArchiveData(long index); long getTrashDataSize(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 4005a88c6f054..209b096b38acf 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -25,7 +25,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.EnumSet; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -50,6 +49,7 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.TrashDataComponent; import org.apache.bookkeeper.mledger.util.CallbackMutex; +import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.Stat; @@ -298,7 +298,7 @@ private byte[] serialize(Map toPersist) { private Map deSerialize(byte[] content) throws InvalidProtocolBufferException { TrashDataComponent component = TrashDataComponent.parseFrom(content); List componentList = component.getComponentList(); - Map result = new HashMap<>(); + Map result = new ConcurrentSkipListMap<>(); for (MLDataFormats.TrashData ele : componentList) { if (ele.hasValue()) { result.put(ele.getKey(), ele.getValue()); @@ -341,13 +341,41 @@ public void triggerDelete() { } @Override - public List getAllArchiveIndex() { - return null; + public CompletableFuture> getAllArchiveIndex() { + return metadataStore.getChildren(buildParentPath()).thenComposeAsync(children -> { + CompletableFuture> future = new CompletableFuture<>(); + if (CollectionUtils.isEmpty(children)) { + future.complete(Collections.emptyList()); + return future; + } + List archiveIndexes = new ArrayList<>(); + for (String ele : children) { + if (!ele.startsWith(ARCHIVE)) { + continue; + } + String indexStr = ele.split(ARCHIVE)[1]; + archiveIndexes.add(Long.parseLong(indexStr)); + } + future.complete(archiveIndexes); + return future; + }, executor.chooseThread(name)); } @Override - public Map getArchiveData(final long index) { - return null; + public CompletableFuture> getArchiveData(final long index) { + return metadataStore.get(buildArchivePath(index)).thenComposeAsync(optResult -> { + CompletableFuture> future = new CompletableFuture<>(); + if (optResult.isPresent()) { + byte[] content = optResult.get().getValue(); + try { + Map result = deSerialize(content); + future.complete(result); + } catch (InvalidProtocolBufferException e) { + future.completeExceptionally(e); + } + } + return future; + }, executor.chooseThread(name)); } @Override @@ -393,16 +421,16 @@ private void updateArchiveDataIfNecessary(final CompletableFuture future) { } - private String buildPath() { + private String buildParentPath() { return PREFIX + name + "/" + type; } private String buildDeletePath() { - return buildPath() + DELETE; + return buildParentPath() + DELETE; } - private String buildArchivePath(long ledgerId) { - return buildPath() + ARCHIVE + ledgerId; + private String buildArchivePath(long index) { + return buildParentPath() + ARCHIVE + index; } //take 1/10 trash to delete, if the size over 10, use 10 to delete. From 2f9acb2670e15568fa4daf76e9536917030290a9 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Tue, 31 May 2022 01:05:10 +0800 Subject: [PATCH 14/43] support trigger config. --- .../mledger/ManagedLedgerConfig.java | 9 +++ .../mledger/impl/ManagedLedgerImpl.java | 4 +- .../mledger/impl/ManagedTrashDisableImpl.java | 81 +++++++++++++++++++ 3 files changed, 92 insertions(+), 2 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index ba884b89ee659..0ca421fb48421 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -82,6 +82,7 @@ public class ManagedLedgerConfig { @Setter private boolean cacheEvictionByMarkDeletedPosition = false; private int archiveDataLimitSize = 500; + private boolean managedTrash = false; public boolean isCreateIfMissing() { return createIfMissing; @@ -691,4 +692,12 @@ public int getArchiveDataLimitSize() { public void setArchiveDataLimitSize(int archiveDataLimitSize) { this.archiveDataLimitSize = archiveDataLimitSize; } + + public boolean isManagedTrash() { + return managedTrash; + } + + public void setManagedTrash(boolean managedTrash) { + this.managedTrash = managedTrash; + } } 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 bf4460f3a8e0f..9c597177adfd0 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 @@ -340,9 +340,9 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper this.managedLedgerInterceptor = config.getManagedLedgerInterceptor(); } this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); - this.managedTrash = + this.managedTrash = config.isManagedTrash() ? new ManagedTrashImpl(ManagedTrash.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, - executor, bookKeeper); + executor, bookKeeper) : new ManagedTrashDisableImpl(); } synchronized void initialize(final ManagedLedgerInitializeLedgerCallback callback, final Object ctx) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java new file mode 100644 index 0000000000000..c03a7ab011dd2 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedTrash; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; + +public class ManagedTrashDisableImpl implements ManagedTrash { + + private static final CompletableFuture COMPLETABLE_FUTURE = CompletableFuture.completedFuture(null); + + @Override + public String name() { + return ""; + } + + @Override + public CompletableFuture initialize() { + return COMPLETABLE_FUTURE; + } + + @Override + public CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) { + return COMPLETABLE_FUTURE; + } + + @Override + public CompletableFuture asyncUpdateTrashData() { + return COMPLETABLE_FUTURE; + } + + @Override + public void triggerDelete() { + } + + @Override + public CompletableFuture> getAllArchiveIndex() { + return (CompletableFuture>) COMPLETABLE_FUTURE; + } + + @Override + public CompletableFuture> getArchiveData(long index) { + return (CompletableFuture>) COMPLETABLE_FUTURE; + } + + @Override + public long getTrashDataSize() { + return 0; + } + + @Override + public long getToArchiveDataSize() { + return 0; + } + + @Override + public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { + callback.closeComplete(ctx); + } +} From 79deefd78bf8cc5127ea19189ad9c896e959f547 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Tue, 31 May 2022 01:21:18 +0800 Subject: [PATCH 15/43] fix check style --- .../org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 9c597177adfd0..1d939bfbe04b4 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 @@ -340,8 +340,8 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper this.managedLedgerInterceptor = config.getManagedLedgerInterceptor(); } this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); - this.managedTrash = config.isManagedTrash() ? - new ManagedTrashImpl(ManagedTrash.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, + this.managedTrash = config.isManagedTrash() + ? new ManagedTrashImpl(ManagedTrash.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, executor, bookKeeper) : new ManagedTrashDisableImpl(); } From 510da889d221a3bf43a688923794c1154fc95330 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Tue, 31 May 2022 10:32:32 +0800 Subject: [PATCH 16/43] compatible --- .../mledger/impl/ManagedLedgerImpl.java | 35 ++++++++++++++++++- 1 file changed, 34 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 1d939bfbe04b4..00716874f6212 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 @@ -2619,7 +2619,20 @@ public void operationComplete(Void result, Stat stat) { ledgersStat = stat; metadataMutex.unlock(); trimmerMutex.unlock(); - executor.executeOrdered(name, safeRun(managedTrash::triggerDelete)); + if (managedTrash instanceof ManagedTrashDisableImpl) { + for (LedgerInfo ls : ledgersToDelete) { + log.info("[{}] Removing ledger {} - size: {}", name, ls.getLedgerId(), ls.getSize()); + asyncDeleteLedger(ls.getLedgerId(), ls); + } + for (LedgerInfo ls : offloadedLedgersToDelete) { + log.info("[{}] Deleting offloaded ledger {} from bookkeeper - size: {}", name, + ls.getLedgerId(), + ls.getSize()); + asyncDeleteLedgerFromBookKeeper(ls.getLedgerId()); + } + } else { + executor.executeOrdered(name, safeRun(managedTrash::triggerDelete)); + } promise.complete(null); } @@ -2641,6 +2654,7 @@ public void operationFailed(MetaStoreException e) { } } + private CompletableFuture asyncUpdateTrashData(Collection deletableLedgerIds, Collection deletableOffloadedLedgerIds) { List> futures = @@ -2787,6 +2801,25 @@ public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) { } } + private void asyncDeleteLedgerFromBookKeeper(long ledgerId) { + asyncDeleteLedger(ledgerId, DEFAULT_LEDGER_DELETE_RETRIES); + } + + private void asyncDeleteLedger(long ledgerId, LedgerInfo info) { + if (!info.getOffloadContext().getBookkeeperDeleted()) { + // only delete if it hasn't been previously deleted for offload + asyncDeleteLedger(ledgerId, DEFAULT_LEDGER_DELETE_RETRIES); + } + + if (info.getOffloadContext().hasUidMsb()) { + UUID uuid = new UUID(info.getOffloadContext().getUidMsb(), info.getOffloadContext().getUidLsb()); + cleanupOffloaded(ledgerId, uuid, + OffloadUtils.getOffloadDriverName(info, config.getLedgerOffloader().getOffloadDriverName()), + OffloadUtils.getOffloadDriverMetadata(info, config.getLedgerOffloader().getOffloadDriverMetadata()), + "Trimming"); + } + } + private void asyncDeleteLedger(long ledgerId, long retry) { if (retry <= 0) { log.warn("[{}] Failed to delete ledger after retries {}", name, ledgerId); From 9dd35c3ebb26d64ff351098b6c2cf13e42d0625b Mon Sep 17 00:00:00 2001 From: horizonzy Date: Tue, 31 May 2022 11:16:51 +0800 Subject: [PATCH 17/43] support config to control is support two phase deletion. --- .../apache/bookkeeper/mledger/ManagedLedgerConfig.java | 10 +++++----- .../bookkeeper/mledger/impl/ManagedLedgerImpl.java | 2 +- .../org/apache/pulsar/broker/ServiceConfiguration.java | 7 +++++++ .../apache/pulsar/broker/service/BrokerService.java | 1 + 4 files changed, 14 insertions(+), 6 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 0ca421fb48421..2878c55383c8d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -82,7 +82,7 @@ public class ManagedLedgerConfig { @Setter private boolean cacheEvictionByMarkDeletedPosition = false; private int archiveDataLimitSize = 500; - private boolean managedTrash = false; + private boolean supportTwoPhaseDeletion = false; public boolean isCreateIfMissing() { return createIfMissing; @@ -693,11 +693,11 @@ public void setArchiveDataLimitSize(int archiveDataLimitSize) { this.archiveDataLimitSize = archiveDataLimitSize; } - public boolean isManagedTrash() { - return managedTrash; + public boolean isSupportTwoPhaseDeletion() { + return supportTwoPhaseDeletion; } - public void setManagedTrash(boolean managedTrash) { - this.managedTrash = managedTrash; + public void setSupportTwoPhaseDeletion(boolean supportTwoPhaseDeletion) { + this.supportTwoPhaseDeletion = supportTwoPhaseDeletion; } } 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 00716874f6212..212f1dfbf5343 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 @@ -340,7 +340,7 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper this.managedLedgerInterceptor = config.getManagedLedgerInterceptor(); } this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); - this.managedTrash = config.isManagedTrash() + this.managedTrash = config.isSupportTwoPhaseDeletion() ? new ManagedTrashImpl(ManagedTrash.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, executor, bookKeeper) : new ManagedTrashDisableImpl(); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 6639fe0cb3513..dc3252097b414 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2514,6 +2514,13 @@ public class ServiceConfiguration implements PulsarConfiguration { ) private int managedLedgerInactiveLedgerRolloverTimeSeconds = 0; + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "Using two phase deletion when delete ledger. (Default value is false)" + ) + private boolean managedLedgerSupportTwoPhaseDeletion; + @FieldContext( category = CATEGORY_STORAGE_ML, doc = "Evicting cache data by the slowest markDeletedPosition or readPosition. " diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 532e41cfbd3c3..699b92ae32f68 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1577,6 +1577,7 @@ public CompletableFuture getManagedLedgerConfig(TopicName t serviceConfig.getManagedLedgerInactiveLedgerRolloverTimeSeconds(), TimeUnit.SECONDS); managedLedgerConfig.setCacheEvictionByMarkDeletedPosition( serviceConfig.isCacheEvictionByMarkDeletedPosition()); + managedLedgerConfig.setSupportTwoPhaseDeletion(serviceConfig.isManagedLedgerSupportTwoPhaseDeletion()); OffloadPoliciesImpl nsLevelOffloadPolicies = (OffloadPoliciesImpl) policies.map(p -> p.offload_policies).orElse(null); From 0f97c20b7a2ecd4e1499b682af4b2ae86d5967eb Mon Sep 17 00:00:00 2001 From: horizonzy Date: Tue, 31 May 2022 11:40:33 +0800 Subject: [PATCH 18/43] code style fix. --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 699b92ae32f68..74608a8c6a4b5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1577,7 +1577,8 @@ public CompletableFuture getManagedLedgerConfig(TopicName t serviceConfig.getManagedLedgerInactiveLedgerRolloverTimeSeconds(), TimeUnit.SECONDS); managedLedgerConfig.setCacheEvictionByMarkDeletedPosition( serviceConfig.isCacheEvictionByMarkDeletedPosition()); - managedLedgerConfig.setSupportTwoPhaseDeletion(serviceConfig.isManagedLedgerSupportTwoPhaseDeletion()); + managedLedgerConfig.setSupportTwoPhaseDeletion( + serviceConfig.isManagedLedgerSupportTwoPhaseDeletion()); OffloadPoliciesImpl nsLevelOffloadPolicies = (OffloadPoliciesImpl) policies.map(p -> p.offload_policies).orElse(null); From 488d2ce559b8b5386876055131407bede60270c5 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Tue, 31 May 2022 16:28:12 +0800 Subject: [PATCH 19/43] make ManagedTrash cover more case. --- .../bookkeeper/mledger/ManagedTrash.java | 4 +- .../mledger/impl/ManagedLedgerImpl.java | 145 +++++++++--------- .../mledger/impl/ManagedTrashImpl.java | 4 +- 3 files changed, 80 insertions(+), 73 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index ebaeba5cccf46..1a7c4ad3f4246 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -31,9 +31,9 @@ public interface ManagedTrash { String SCHEMA = "schema"; - String DELETABLE_LEDGER_SUFFIX = "DL"; + String DELETABLE_LEDGER = "DL"; - String DELETABLE_OFFLOADED_LEDGER_SUFFIX = "DOL"; + String DELETABLE_OFFLOADED_LEDGER = "DOL"; String name(); 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 212f1dfbf5343..91e576282a424 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 @@ -2578,7 +2578,7 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { .filter(ls -> ls.getOffloadContext().hasUidMsb() && ls.getOffloadContext().getComplete()) .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); - CompletableFuture updateTrashFuture = + CompletableFuture updateTrashFuture = asyncUpdateTrashData(deletableLedgers, deletableOffloadedLedgers); updateTrashFuture.thenAccept(ignore -> { for (LedgerInfo ls : ledgersToDelete) { @@ -2654,20 +2654,19 @@ public void operationFailed(MetaStoreException e) { } } - - private CompletableFuture asyncUpdateTrashData(Collection deletableLedgerIds, + private CompletableFuture asyncUpdateTrashData(Collection deletableLedgerIds, Collection deletableOffloadedLedgerIds) { List> futures = new ArrayList<>(deletableLedgerIds.size() + deletableOffloadedLedgerIds.size()); for (Long ledgerId : deletableLedgerIds) { - futures.add(managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.DELETABLE_LEDGER_SUFFIX)); + futures.add(managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.DELETABLE_LEDGER)); } for (Long ledgerId : deletableOffloadedLedgerIds) { futures.add(managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), - ManagedTrash.DELETABLE_OFFLOADED_LEDGER_SUFFIX)); + ManagedTrash.DELETABLE_OFFLOADED_LEDGER)); } - futures.add(managedTrash.asyncUpdateTrashData()); - return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])); + return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])) + .thenCompose(ignore -> managedTrash.asyncUpdateTrashData()); } /** @@ -3049,10 +3048,7 @@ private void offloadLoop(CompletableFuture promise, Queue prepareLedgerInfoForOffloaded(long ledgerId, UUID uuid, String offloadDriverName, - Map offloadDriverMetadata) { + Map offloadDriverMetadata) { log.info("[{}] Preparing metadata to offload ledger {} with uuid {}", name, ledgerId, uuid); return transformLedgerInfo(ledgerId, - (oldInfo) -> { - if (oldInfo.getOffloadContext().hasUidMsb()) { - UUID oldUuid = new UUID(oldInfo.getOffloadContext().getUidMsb(), - oldInfo.getOffloadContext().getUidLsb()); - log.info("[{}] Found previous offload attempt for ledger {}, uuid {}" - + ", cleaning up", name, ledgerId, uuid); - cleanupOffloaded( - ledgerId, - oldUuid, - OffloadUtils.getOffloadDriverName(oldInfo, - config.getLedgerOffloader().getOffloadDriverName()), - OffloadUtils.getOffloadDriverMetadata(oldInfo, - config.getLedgerOffloader().getOffloadDriverMetadata()), - "Previous failed offload"); - } - LedgerInfo.Builder builder = oldInfo.toBuilder(); - builder.getOffloadContextBuilder() - .setUidMsb(uuid.getMostSignificantBits()) - .setUidLsb(uuid.getLeastSignificantBits()); - OffloadUtils.setOffloadDriverMetadata( - builder, - offloadDriverName, - offloadDriverMetadata - ); - return builder.build(); - }) - .whenComplete((result, exception) -> { + (oldInfo) -> { + if (oldInfo.getOffloadContext().hasUidMsb()) { + UUID oldUuid = new UUID(oldInfo.getOffloadContext().getUidMsb(), + oldInfo.getOffloadContext().getUidLsb()); + log.info("[{}] Found previous offload attempt for ledger {}, uuid {} old uuid {}" + + ", cleaning up", name, ledgerId, uuid, oldUuid); + if (managedTrash instanceof ManagedTrashDisableImpl) { + cleanupOffloaded( + ledgerId, + oldUuid, + OffloadUtils.getOffloadDriverName(oldInfo, + config.getLedgerOffloader().getOffloadDriverName()), + OffloadUtils.getOffloadDriverMetadata(oldInfo, + config.getLedgerOffloader().getOffloadDriverMetadata()), + "Previous failed offload"); + } else { + managedTrash.appendLedgerTrashData(ledgerId, oldInfo, + ManagedTrash.DELETABLE_OFFLOADED_LEDGER) + .thenAccept(ignore -> managedTrash.asyncUpdateTrashData()); + } + } + LedgerInfo.Builder builder = oldInfo.toBuilder(); + builder.getOffloadContextBuilder() + .setUidMsb(uuid.getMostSignificantBits()) + .setUidLsb(uuid.getLeastSignificantBits()); + OffloadUtils.setOffloadDriverMetadata( + builder, + offloadDriverName, + offloadDriverMetadata + ); + return builder.build(); + }) + .whenComplete((result, exception) -> { if (exception != null) { log.warn("[{}] Failed to prepare ledger {} for offload, uuid {}", - name, ledgerId, uuid, exception); + name, ledgerId, uuid, exception); } else { log.info("[{}] Metadata prepared for offload of ledger {} with uuid {}", name, ledgerId, uuid); } @@ -3203,38 +3205,38 @@ private CompletableFuture prepareLedgerInfoForOffloaded(long ledgerId, UUI private CompletableFuture completeLedgerInfoForOffloaded(long ledgerId, UUID uuid) { log.info("[{}] Completing metadata for offload of ledger {} with uuid {}", name, ledgerId, uuid); return transformLedgerInfo(ledgerId, - (oldInfo) -> { - UUID existingUuid = new UUID(oldInfo.getOffloadContext().getUidMsb(), - oldInfo.getOffloadContext().getUidLsb()); - if (existingUuid.equals(uuid)) { - LedgerInfo.Builder builder = oldInfo.toBuilder(); - builder.getOffloadContextBuilder() - .setTimestamp(clock.millis()) - .setComplete(true); - - String driverName = OffloadUtils.getOffloadDriverName( - oldInfo, config.getLedgerOffloader().getOffloadDriverName()); - Map driverMetadata = OffloadUtils.getOffloadDriverMetadata( - oldInfo, config.getLedgerOffloader().getOffloadDriverMetadata()); - OffloadUtils.setOffloadDriverMetadata( - builder, - driverName, - driverMetadata - ); - return builder.build(); - } else { - throw new OffloadConflict( - "Existing UUID(" + existingUuid + ") in metadata for offload" - + " of ledgerId " + ledgerId + " does not match the UUID(" + uuid - + ") for the offload we are trying to complete"); - } - }) - .whenComplete((result, exception) -> { + (oldInfo) -> { + UUID existingUuid = new UUID(oldInfo.getOffloadContext().getUidMsb(), + oldInfo.getOffloadContext().getUidLsb()); + if (existingUuid.equals(uuid)) { + LedgerInfo.Builder builder = oldInfo.toBuilder(); + builder.getOffloadContextBuilder() + .setTimestamp(clock.millis()) + .setComplete(true); + + String driverName = OffloadUtils.getOffloadDriverName( + oldInfo, config.getLedgerOffloader().getOffloadDriverName()); + Map driverMetadata = OffloadUtils.getOffloadDriverMetadata( + oldInfo, config.getLedgerOffloader().getOffloadDriverMetadata()); + OffloadUtils.setOffloadDriverMetadata( + builder, + driverName, + driverMetadata + ); + return builder.build(); + } else { + throw new OffloadConflict( + "Existing UUID(" + existingUuid + ") in metadata for offload" + + " of ledgerId " + ledgerId + " does not match the UUID(" + uuid + + ") for the offload we are trying to complete"); + } + }) + .whenComplete((result, exception) -> { if (exception == null) { log.info("[{}] End Offload. ledger={}, uuid={}", name, ledgerId, uuid); } else { log.warn("[{}] Failed to complete offload of ledger {}, uuid {}", - name, ledgerId, uuid, exception); + name, ledgerId, uuid, exception); } }); } @@ -3864,7 +3866,12 @@ protected boolean checkAndCompleteLedgerOpTask(int rc, LedgerHandle lh, Object c } else { if (rc == BKException.Code.OK) { log.warn("[{}]-{} ledger creation timed-out, deleting ledger", this.name, lh.getId()); - asyncDeleteLedger(lh.getId(), DEFAULT_LEDGER_DELETE_RETRIES); + if (managedTrash instanceof ManagedTrashDisableImpl) { + asyncDeleteLedger(lh.getId(), DEFAULT_LEDGER_DELETE_RETRIES); + } else { + managedTrash.appendLedgerTrashData(lh.getId(), null, ManagedTrash.DELETABLE_LEDGER). + thenAccept(ignore -> managedTrash.asyncUpdateTrashData()); + } } return true; } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 209b096b38acf..6faf41836d052 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -697,11 +697,11 @@ private String transferToTrashKey() { } private boolean isLedger() { - return DELETABLE_LEDGER_SUFFIX.equals(suffix); + return DELETABLE_LEDGER.equals(suffix); } private boolean isOffloadLedger() { - return DELETABLE_OFFLOADED_LEDGER_SUFFIX.equals(suffix); + return DELETABLE_OFFLOADED_LEDGER.equals(suffix); } } From 234dd94d370313a49a2e83cb5f602f058ffa686d Mon Sep 17 00:00:00 2001 From: horizonzy Date: Tue, 31 May 2022 17:28:38 +0800 Subject: [PATCH 20/43] make triggerDeleteInBackground to interface. --- .../java/org/apache/bookkeeper/mledger/ManagedTrash.java | 2 +- .../apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 2 +- .../apache/bookkeeper/mledger/impl/ManagedTrashImpl.java | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 1a7c4ad3f4246..436e6bfd7f445 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -43,7 +43,7 @@ public interface ManagedTrash { CompletableFuture asyncUpdateTrashData(); - void triggerDelete(); + void triggerDeleteInBackground(); CompletableFuture> getAllArchiveIndex(); 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 91e576282a424..6c9589092d998 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 @@ -2631,7 +2631,7 @@ public void operationComplete(Void result, Stat stat) { asyncDeleteLedgerFromBookKeeper(ls.getLedgerId()); } } else { - executor.executeOrdered(name, safeRun(managedTrash::triggerDelete)); + managedTrash.triggerDeleteInBackground(); } promise.complete(null); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 6faf41836d052..af9d8b00011f1 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -309,12 +309,12 @@ private Map deSerialize(byte[] content) throws InvalidProtoc return result; } - private void triggerDeleteInBackground() { + @Override + public void triggerDeleteInBackground() { executor.executeOrdered(name, safeRun(this::triggerDelete)); } - @Override - public void triggerDelete() { + private void triggerDelete() { State state = STATE_UPDATER.get(this); if (state != State.INITIALIZED) { log.warn("[{}] is not initialized, current state: {}", name(), state); From cfb950dc0c4deb7818e71c697fc8aad1546f9d27 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Tue, 31 May 2022 17:29:49 +0800 Subject: [PATCH 21/43] make triggerDeleteInBackground to interface. --- .../apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index c03a7ab011dd2..7d794c552cf7d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -51,7 +51,7 @@ public CompletableFuture asyncUpdateTrashData() { } @Override - public void triggerDelete() { + public void triggerDeleteInBackground() { } @Override From 428bf11ec97cc802f16e8f7e7c1f5e9a805d603e Mon Sep 17 00:00:00 2001 From: horizonzy Date: Wed, 1 Jun 2022 10:59:02 +0800 Subject: [PATCH 22/43] fix ignore uncompleted offload ledger problem. --- .../mledger/impl/ManagedLedgerImpl.java | 36 ++++++++++--------- 1 file changed, 20 insertions(+), 16 deletions(-) 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 6c9589092d998..787ad7089a64d 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 @@ -2500,10 +2500,6 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { } break; } - //The offload process not completed, shouldn't delete it. - if (ls.hasOffloadContext() && !ls.getOffloadContext().getComplete()) { - continue; - } // if truncate, all ledgers besides currentLedger are going to be deleted if (isTruncate) { if (log.isDebugEnabled()) { @@ -2575,7 +2571,7 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { // Mark deletable offloaded ledgers Set deletableOffloadedLedgers = ledgersToDelete.stream() - .filter(ls -> ls.getOffloadContext().hasUidMsb() && ls.getOffloadContext().getComplete()) + .filter(ls -> ls.getOffloadContext().hasUidMsb()) .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); CompletableFuture updateTrashFuture = @@ -3036,23 +3032,31 @@ private void offloadLoop(CompletableFuture promise, Queue driverMetadata = config.getLedgerOffloader().getOffloadDriverMetadata(); prepareLedgerInfoForOffloaded(ledgerId, uuid, driverName, driverMetadata) - .thenCompose((ignore) -> getLedgerHandle(ledgerId)) - .thenCompose(readHandle -> config.getLedgerOffloader().offload(readHandle, uuid, extraMetadata)) - .thenCompose((ignore) -> { + .thenCompose((ignore) -> getLedgerHandle(ledgerId)) + .thenCompose(readHandle -> config.getLedgerOffloader().offload(readHandle, uuid, extraMetadata)) + .thenCompose((ignore) -> { return Retries.run(Backoff.exponentialJittered(TimeUnit.SECONDS.toMillis(1), - TimeUnit.SECONDS.toHours(1)).limit(10), - FAIL_ON_CONFLICT, - () -> completeLedgerInfoForOffloaded(ledgerId, uuid), - scheduledExecutor, name) - .whenComplete((ignore2, exception) -> { + TimeUnit.SECONDS.toHours(1)).limit(10), + FAIL_ON_CONFLICT, + () -> completeLedgerInfoForOffloaded(ledgerId, uuid), + scheduledExecutor, name) + .whenComplete((ignore2, exception) -> { if (exception != null) { log.error("[{}] Failed to offload data for the ledgerId {}", name, ledgerId, exception); //not delete offload ledger, it will delete at next offload. + if (managedTrash instanceof ManagedTrashDisableImpl) { + cleanupOffloaded(ledgerId, uuid, driverName, driverMetadata, + "Metastore failure"); + } else { + managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), + ManagedTrash.DELETABLE_OFFLOADED_LEDGER) + .thenAccept(ignore3 -> managedTrash.asyncUpdateTrashData()); + } } }); }) - .whenComplete((ignore, exception) -> { + .whenComplete((ignore, exception) -> { if (exception != null) { lastOffloadFailureTimestamp = System.currentTimeMillis(); log.warn("[{}] Exception occurred for ledgerId {} timestamp {} during offload", name, @@ -3070,8 +3074,8 @@ private void offloadLoop(CompletableFuture promise, Queue Date: Wed, 1 Jun 2022 21:40:40 +0800 Subject: [PATCH 23/43] when close managedLedger, close managedTrash together. --- .../bookkeeper/mledger/ManagedTrash.java | 7 +- .../mledger/impl/ManagedLedgerImpl.java | 126 ++++++++---- .../mledger/impl/ManagedTrashDisableImpl.java | 3 +- .../mledger/impl/ManagedTrashImpl.java | 191 +++++++----------- 4 files changed, 164 insertions(+), 163 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 436e6bfd7f445..4e7aa534c6a23 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -31,15 +31,16 @@ public interface ManagedTrash { String SCHEMA = "schema"; - String DELETABLE_LEDGER = "DL"; + String LEDGER = "L"; - String DELETABLE_OFFLOADED_LEDGER = "DOL"; + String OFFLOADED_LEDGER = "OL"; String name(); CompletableFuture initialize(); - CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type); + void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) + throws ManagedLedgerException; CompletableFuture asyncUpdateTrashData(); 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 787ad7089a64d..ffd72882d1e16 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 @@ -1412,38 +1412,64 @@ public synchronized void asyncClose(final CloseCallback callback, final Object c LedgerHandle lh = currentLedger; - if (lh == null) { - // No ledger to close, proceed with next step - closeAllCursors(callback, ctx); - return; - } + List> futures = new ArrayList<>(); - if (log.isDebugEnabled()) { - log.debug("[{}] Closing current writing ledger {}", name, lh.getId()); - } + CompletableFuture closeTrashFuture = new CompletableFuture<>(); + futures.add(closeTrashFuture); - mbean.startDataLedgerCloseOp(); - lh.asyncClose((rc, lh1, ctx1) -> { - if (log.isDebugEnabled()) { - log.debug("[{}] Close complete for ledger {}: rc = {}", name, lh.getId(), rc); + managedTrash.asyncClose(new CloseCallback() { + @Override + public void closeComplete(Object ctx) { + closeTrashFuture.complete(null); } - mbean.endDataLedgerCloseOp(); - if (rc != BKException.Code.OK) { - callback.closeFailed(createManagedLedgerException(rc), ctx); - return; + + @Override + public void closeFailed(ManagedLedgerException exception, Object ctx) { + closeTrashFuture.completeExceptionally(exception); } + }, ctx); - ledgerCache.forEach((ledgerId, readHandle) -> { - invalidateReadHandle(ledgerId); - }); - closeAllCursors(callback, ctx); - }, null); + CompletableFuture closeCursorFuture = new CompletableFuture<>(); + futures.add(closeCursorFuture); + if (lh == null) { + // No ledger to close, proceed with next step + closeAllCursors(closeCursorFuture); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Closing current writing ledger {}", name, lh.getId()); + } + + mbean.startDataLedgerCloseOp(); + lh.asyncClose((rc, lh1, ctx1) -> { + if (log.isDebugEnabled()) { + log.debug("[{}] Close complete for ledger {}: rc = {}", name, lh.getId(), rc); + } + mbean.endDataLedgerCloseOp(); + if (rc != BKException.Code.OK) { + closeCursorFuture.completeExceptionally(createManagedLedgerException(rc)); + return; + } + + ledgerCache.forEach((ledgerId, readHandle) -> { + invalidateReadHandle(ledgerId); + }); + + closeAllCursors(closeCursorFuture); + }, null); + } + + FutureUtil.waitForAll(futures).thenAccept(ignore -> { + callback.closeComplete(ctx); + }).exceptionally(e -> { + callback.closeFailed(ManagedLedgerException.getManagedLedgerException(e), ctx); + return null; + }); } - private void closeAllCursors(CloseCallback callback, final Object ctx) { + private void closeAllCursors(CompletableFuture future) { // Close all cursors in parallel List> futures = Lists.newArrayList(); for (ManagedCursor cursor : cursors) { @@ -1453,11 +1479,12 @@ private void closeAllCursors(CloseCallback callback, final Object ctx) { } Futures.waitForAll(futures) - .thenRun(() -> callback.closeComplete(ctx)) + .thenRun(() -> future.complete(null)) .exceptionally(exception -> { - callback.closeFailed(ManagedLedgerException.getManagedLedgerException(exception.getCause()), ctx); - return null; - }); + future.completeExceptionally( + ManagedLedgerException.getManagedLedgerException(exception.getCause())); + return null; + }); } // ////////////////////////////////////////////////////////////////////// @@ -2651,18 +2678,21 @@ public void operationFailed(MetaStoreException e) { } private CompletableFuture asyncUpdateTrashData(Collection deletableLedgerIds, - Collection deletableOffloadedLedgerIds) { - List> futures = - new ArrayList<>(deletableLedgerIds.size() + deletableOffloadedLedgerIds.size()); - for (Long ledgerId : deletableLedgerIds) { - futures.add(managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.DELETABLE_LEDGER)); - } - for (Long ledgerId : deletableOffloadedLedgerIds) { - futures.add(managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), - ManagedTrash.DELETABLE_OFFLOADED_LEDGER)); + Collection deletableOffloadedLedgerIds) { + CompletableFuture future = new CompletableFuture<>(); + try { + for (Long ledgerId : deletableLedgerIds) { + managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.LEDGER); + } + for (Long ledgerId : deletableOffloadedLedgerIds) { + managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), + ManagedTrash.OFFLOADED_LEDGER); + } + future.complete(null); + } catch (ManagedLedgerException e) { + future.completeExceptionally(e); } - return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])) - .thenCompose(ignore -> managedTrash.asyncUpdateTrashData()); + return future.thenCompose(ignore -> managedTrash.asyncUpdateTrashData()); } /** @@ -3049,9 +3079,13 @@ private void offloadLoop(CompletableFuture promise, Queue managedTrash.asyncUpdateTrashData()); + try { + managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), + ManagedTrash.OFFLOADED_LEDGER); + managedTrash.asyncUpdateTrashData(); + } catch (ManagedLedgerException e) { + log.warn("[{}]-{} Failed to append trash data.", this.name, ledgerId); + } } } }); @@ -3181,8 +3215,8 @@ private CompletableFuture prepareLedgerInfoForOffloaded(long ledgerId, UUI "Previous failed offload"); } else { managedTrash.appendLedgerTrashData(ledgerId, oldInfo, - ManagedTrash.DELETABLE_OFFLOADED_LEDGER) - .thenAccept(ignore -> managedTrash.asyncUpdateTrashData()); + ManagedTrash.OFFLOADED_LEDGER); + managedTrash.asyncUpdateTrashData(); } } LedgerInfo.Builder builder = oldInfo.toBuilder(); @@ -3873,8 +3907,12 @@ protected boolean checkAndCompleteLedgerOpTask(int rc, LedgerHandle lh, Object c if (managedTrash instanceof ManagedTrashDisableImpl) { asyncDeleteLedger(lh.getId(), DEFAULT_LEDGER_DELETE_RETRIES); } else { - managedTrash.appendLedgerTrashData(lh.getId(), null, ManagedTrash.DELETABLE_LEDGER). - thenAccept(ignore -> managedTrash.asyncUpdateTrashData()); + try { + managedTrash.appendLedgerTrashData(lh.getId(), null, ManagedTrash.LEDGER); + managedTrash.asyncUpdateTrashData(); + } catch (ManagedLedgerException e) { + log.warn("[{}]-{} Failed to append trash data.", this.name, lh.getId()); + } } } return true; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index 7d794c552cf7d..0824fcebd769b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -41,8 +41,7 @@ public CompletableFuture initialize() { } @Override - public CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) { - return COMPLETABLE_FUTURE; + public void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) { } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index af9d8b00011f1..6539d93a4af96 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -68,9 +68,11 @@ public class ManagedTrashImpl implements ManagedTrash { private static final String PREFIX = BASE_NODE + "/"; - private static final String DELETE = "/delete"; + private static final String DELETE_SUFFIX = "/delete"; - private static final String ARCHIVE = "/archive-"; + private static final String ARCHIVE = "archive-"; + + private static final String ARCHIVE_SUFFIX = "/" + ARCHIVE; private static final String TRASH_KEY_SEPARATOR = "-"; @@ -86,12 +88,12 @@ public class ManagedTrashImpl implements ManagedTrash { protected volatile ManagedTrashImpl.State state = null; //key:ledgerId value:storageContext + //todo: key 换成对象 + //todo 未达到 archiveLimit 的 trashData 中 leftRetryCount == 0 的数据是否需要单独一个节点维护数据 private NavigableMap trashData = new ConcurrentSkipListMap<>(); private final AtomicInteger toArchiveCount = new AtomicInteger(); - private final CallbackMutex trashMutex = new CallbackMutex(); - private final CallbackMutex deleteMutex = new CallbackMutex(); private final CallbackMutex trashPersistMutex = new CallbackMutex(); @@ -159,21 +161,15 @@ public CompletableFuture initialize() { try { trashData.putAll(deSerialize(value)); deleteStat = res.get().getStat(); - calculateArchiveCount().whenComplete((res1, e1) -> { - if (e1 != null) { - future.completeExceptionally(getException(e1)); - return; - } - toArchiveCount.set(res1); - future.complete(null); - checkTrashPersistTask = - scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, - TimeUnit.MINUTES); - STATE_UPDATER.set(this, State.INITIALIZED); - triggerDeleteInBackground(); - }); + toArchiveCount.set(calculateArchiveCount()); + future.complete(null); + checkTrashPersistTask = + scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, + TimeUnit.MINUTES); + STATE_UPDATER.set(this, State.INITIALIZED); + triggerDeleteInBackground(); } catch (InvalidProtocolBufferException exc) { - future.completeExceptionally(getException(exc)); + future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(exc)); } }, scheduledExecutor.chooseThread(name)); return future; @@ -185,77 +181,33 @@ private void persistTrashIfNecessary() { } } - private CompletableFuture calculateArchiveCount() { - CompletableFuture future = new CompletableFuture<>(); - calculateArchiveCountInBackground(future); - return future; - } - - private void calculateArchiveCountInBackground(CompletableFuture future) { - executor.executeOrdered(name, safeRun(() -> internalCalculateArchiveCount(future))); - } - - private void internalCalculateArchiveCount(CompletableFuture future) { - if (!trashMutex.tryLock()) { - scheduledExecutor.schedule(safeRun(() -> calculateArchiveCountInBackground(future)), 100, - TimeUnit.MILLISECONDS); - return; - } - try { - int toArchiveCount = 0; - for (Map.Entry entry : trashData.entrySet()) { - if (!entry.getKey().startsWith("0")) { - break; - } - toArchiveCount++; + private int calculateArchiveCount() { + int toArchiveCount = 0; + for (Map.Entry entry : trashData.entrySet()) { + if (!entry.getKey().startsWith("0")) { + break; } - future.complete(toArchiveCount); - } finally { - trashMutex.unlock(); + toArchiveCount++; } + return toArchiveCount; } @Override - public CompletableFuture appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) { - CompletableFuture future = new CompletableFuture<>(); - if (context == null) { - context = EMPTY_LEDGER_INFO; - } - appendTrashData(buildKey(RETRY_COUNT, ledgerId, type), context, future); - return future; - } - - private static ManagedLedgerException.MetaStoreException getException(Throwable t) { - if (t.getCause() instanceof MetadataStoreException.BadVersionException) { - return new ManagedLedgerException.BadVersionException(t.getMessage()); - } else { - return new ManagedLedgerException.MetaStoreException(t); - } - } - - public void appendInBackground(final String key, final LedgerInfo context, final CompletableFuture future) { - executor.executeOrdered(name, safeRun(() -> appendTrashData(key, context, future))); - } - - private void appendTrashData(final String key, final LedgerInfo context, final CompletableFuture future) { + public void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) + throws ManagedLedgerException { State state = STATE_UPDATER.get(this); if (state != State.INITIALIZED) { - future.completeExceptionally(getException(new IllegalStateException( - String.format("[%s] is not initialized, current state: %s", name(), state)))); - return; - } - if (!trashMutex.tryLock()) { - scheduledExecutor.schedule(safeRun(() -> appendInBackground(key, context, future)), 100, - TimeUnit.MILLISECONDS); - return; + throw ManagedLedgerException.getManagedLedgerException(new IllegalStateException( + String.format("[%s] is not initialized, current state: %s", name(), state))); } - try { - trashData.put(key, context); - managedTrashMXBean.increaseTotalNumberOfDeleteLedgers(); - trashIsDirty = true; - } finally { - trashMutex.unlock(); + if (context == null) { + context = EMPTY_LEDGER_INFO; } + String key = buildKey(RETRY_COUNT, ledgerId, type); + trashData.put(key, context); + managedTrashMXBean.increaseTotalNumberOfDeleteLedgers(); + trashIsDirty = true; + } @Override @@ -264,7 +216,7 @@ public CompletableFuture asyncUpdateTrashData() { CompletableFuture future = new CompletableFuture<>(); State state = STATE_UPDATER.get(this); if (state != State.INITIALIZED) { - future.completeExceptionally(getException(new IllegalStateException( + future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(new IllegalStateException( String.format("[%s] is not initialized, current state: %s", name(), state)))); return future; } @@ -272,7 +224,7 @@ public CompletableFuture asyncUpdateTrashData() { deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion())) .whenCompleteAsync((res, e) -> { if (e != null) { - future.completeExceptionally(getException(e)); + future.completeExceptionally(getMetaStoreException(e)); return; } deleteStat = res; @@ -323,18 +275,13 @@ private void triggerDelete() { if (!deleteMutex.tryLock()) { return; } - if (!trashMutex.tryLock()) { - scheduledExecutor.schedule(this::triggerDeleteInBackground, 100, TimeUnit.MILLISECONDS); - deleteMutex.unlock(); - return; - } - List toDelete = getToDeleteData(); + List toDelete = getToDeleteData(); + if (toDelete.size() == 0) { deleteMutex.unlock(); - trashMutex.unlock(); return; } - for (TrashDeleteHelper delHelper : toDelete) { + for (TrashDataKey delHelper : toDelete) { //unlock in asyncDeleteTrash asyncDeleteTrash(delHelper); } @@ -390,6 +337,10 @@ public long getToArchiveDataSize() { @Override public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { + if (STATE_UPDATER.get(this) == State.Closed) { + callback.closeComplete(ctx); + return; + } if (checkTrashPersistTask != null) { checkTrashPersistTask.cancel(true); checkTrashPersistTask = null; @@ -420,21 +371,20 @@ private void updateArchiveDataIfNecessary(final CompletableFuture future) { asyncUpdateArchiveData(future); } - private String buildParentPath() { - return PREFIX + name + "/" + type; + return PREFIX + type + "/" + name; } private String buildDeletePath() { - return buildParentPath() + DELETE; + return buildParentPath() + DELETE_SUFFIX; } private String buildArchivePath(long index) { - return buildParentPath() + ARCHIVE + index; + return buildParentPath() + ARCHIVE_SUFFIX + index; } //take 1/10 trash to delete, if the size over 10, use 10 to delete. - private List getToDeleteData() { + private List getToDeleteData() { if (trashData.size() == 0) { return Collections.emptyList(); } @@ -445,9 +395,9 @@ private List getToDeleteData() { if (batchSize == 0) { batchSize = 1; } - List toDelete = new ArrayList<>(batchSize); + List toDelete = new ArrayList<>(batchSize); for (Map.Entry entry : trashData.descendingMap().entrySet()) { - TrashDeleteHelper delHelper = TrashDeleteHelper.build(entry.getKey(), entry.getValue()); + TrashDataKey delHelper = TrashDataKey.build(entry.getKey(), entry.getValue()); //if last retryCount is zero, the before data retryCount is zero too. if (delHelper.retryCount == 0) { break; @@ -462,6 +412,13 @@ private List getToDeleteData() { private void asyncUpdateArchiveData(CompletableFuture future) { log.info("[{}] Start async update archive data", name()); + + State state = STATE_UPDATER.get(this); + if (state != State.INITIALIZED) { + future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(new IllegalStateException( + String.format("[%s] is not initialized, current state: %s", name(), state)))); + return; + } //transaction operation NavigableMap persistDelete = new ConcurrentSkipListMap<>(); NavigableMap persistArchive = new ConcurrentSkipListMap<>(); @@ -486,7 +443,7 @@ private void asyncUpdateArchiveData(CompletableFuture future) { //build archive persist operation Map.Entry lastEntry = persistArchive.lastEntry(); OpPut opArchivePersist = - new OpPut(buildArchivePath(TrashDeleteHelper.build(lastEntry.getKey()).ledgerId), + new OpPut(buildArchivePath(TrashDataKey.build(lastEntry.getKey()).ledgerId), serialize(persistArchive), Optional.of(-1L), EnumSet.noneOf(CreateOption.class)); txOps.add(opDeletePersist); txOps.add(opArchivePersist); @@ -495,13 +452,13 @@ private void asyncUpdateArchiveData(CompletableFuture future) { opDeletePersist.getFuture().whenCompleteAsync((res, e) -> { if (e != null) { log.error("[{}] Persist trash data failed.", name(), e); - future.completeExceptionally(getException(e)); + future.completeExceptionally(getMetaStoreException(e)); return; } opArchivePersist.getFuture().whenCompleteAsync((res1, e1) -> { if (e1 != null) { log.error("[{}] Persist archive data failed.", name(), e1); - future.completeExceptionally(getException(e1)); + future.completeExceptionally(getMetaStoreException(e1)); return; } deleteStat = res; @@ -519,7 +476,7 @@ private static String buildKey(int retryCount, long ledgerId, String suffix) { + suffix; } - private void asyncDeleteTrash(TrashDeleteHelper delHelper) { + private void asyncDeleteTrash(TrashDataKey delHelper) { if (delHelper.isLedger()) { asyncDeleteLedger(delHelper.ledgerId, new AsyncCallbacks.DeleteLedgerCallback() { @Override @@ -548,7 +505,7 @@ public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { } } - private void onDeleteSuccess(TrashDeleteHelper delHelper) { + private void onDeleteSuccess(TrashDataKey delHelper) { try { String key = delHelper.transferToTrashKey(); if (log.isDebugEnabled()) { @@ -565,14 +522,13 @@ private void onDeleteSuccess(TrashDeleteHelper delHelper) { } finally { boolean continueToDelete = continueToDelete(); deleteMutex.unlock(); - trashMutex.unlock(); if (continueToDelete) { triggerDeleteInBackground(); } } } - private void onDeleteFailed(TrashDeleteHelper delHelper) { + private void onDeleteFailed(TrashDataKey delHelper) { try { //override old key String key = delHelper.transferToTrashKey(); @@ -597,7 +553,6 @@ private void onDeleteFailed(TrashDeleteHelper delHelper) { } finally { boolean continueToDelete = continueToDelete(); deleteMutex.unlock(); - trashMutex.unlock(); if (continueToDelete) { triggerDeleteInBackground(); } @@ -605,7 +560,7 @@ private void onDeleteFailed(TrashDeleteHelper delHelper) { } private boolean continueToDelete() { - TrashDeleteHelper delHelper = TrashDeleteHelper.build(trashData.lastEntry().getKey()); + TrashDataKey delHelper = TrashDataKey.build(trashData.lastEntry().getKey()); return delHelper.retryCount > 0; } @@ -660,8 +615,16 @@ private void asyncDeleteOffloadedLedger(long ledgerId, LedgerInfo info, } } + private static ManagedLedgerException.MetaStoreException getMetaStoreException(Throwable t) { + if (t.getCause() instanceof MetadataStoreException.BadVersionException) { + return new ManagedLedgerException.BadVersionException(t.getMessage()); + } else { + return new ManagedLedgerException.MetaStoreException(t); + } + } + - private static class TrashDeleteHelper { + private static class TrashDataKey { private final int retryCount; @@ -671,25 +634,25 @@ private static class TrashDeleteHelper { private final LedgerInfo context; - public TrashDeleteHelper(int retryCount, long ledgerId, String suffix, LedgerInfo context) { + public TrashDataKey(int retryCount, long ledgerId, String suffix, LedgerInfo context) { this.retryCount = retryCount; this.ledgerId = ledgerId; this.suffix = suffix; this.context = context; } - public static TrashDeleteHelper build(String key) { + public static TrashDataKey build(String key) { String[] split = key.split(TRASH_KEY_SEPARATOR); int retryCont = Integer.parseInt(split[0]); long ledgerId = Long.parseLong(split[1]); - return new TrashDeleteHelper(retryCont, ledgerId, split[2], null); + return new TrashDataKey(retryCont, ledgerId, split[2], null); } - public static TrashDeleteHelper build(String key, LedgerInfo context) { + public static TrashDataKey build(String key, LedgerInfo context) { String[] split = key.split(TRASH_KEY_SEPARATOR); int retryCont = Integer.parseInt(split[0]); long ledgerId = Long.parseLong(split[1]); - return new TrashDeleteHelper(retryCont, ledgerId, split[2], context); + return new TrashDataKey(retryCont, ledgerId, split[2], context); } private String transferToTrashKey() { @@ -697,11 +660,11 @@ private String transferToTrashKey() { } private boolean isLedger() { - return DELETABLE_LEDGER.equals(suffix); + return LEDGER.equals(suffix); } private boolean isOffloadLedger() { - return DELETABLE_OFFLOADED_LEDGER.equals(suffix); + return OFFLOADED_LEDGER.equals(suffix); } } From 02c78678815b2a79099a8bd629a1c4a9422e436a Mon Sep 17 00:00:00 2001 From: horizonzy Date: Thu, 2 Jun 2022 09:52:19 +0800 Subject: [PATCH 24/43] use object to take place of string on trashData key. --- .../bookkeeper/mledger/ManagedTrash.java | 2 +- .../mledger/impl/ManagedTrashDisableImpl.java | 4 +- .../mledger/impl/ManagedTrashImpl.java | 196 +++++++++++------- 3 files changed, 124 insertions(+), 78 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 4e7aa534c6a23..16a92274941e9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -48,7 +48,7 @@ void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) CompletableFuture> getAllArchiveIndex(); - CompletableFuture> getArchiveData(long index); + CompletableFuture> getArchiveData(long index); long getTrashDataSize(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index 0824fcebd769b..50798105fb1b5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -59,8 +59,8 @@ public CompletableFuture> getAllArchiveIndex() { } @Override - public CompletableFuture> getArchiveData(long index) { - return (CompletableFuture>) COMPLETABLE_FUTURE; + public CompletableFuture> getArchiveData(long index) { + return (CompletableFuture>) COMPLETABLE_FUTURE; } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 6539d93a4af96..ee059b3757ae7 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.EnumSet; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -87,10 +88,9 @@ public class ManagedTrashImpl implements ManagedTrash { protected volatile ManagedTrashImpl.State state = null; - //key:ledgerId value:storageContext - //todo: key 换成对象 + private NavigableMap trashData = new ConcurrentSkipListMap<>(); + //todo 未达到 archiveLimit 的 trashData 中 leftRetryCount == 0 的数据是否需要单独一个节点维护数据 - private NavigableMap trashData = new ConcurrentSkipListMap<>(); private final AtomicInteger toArchiveCount = new AtomicInteger(); @@ -183,8 +183,8 @@ private void persistTrashIfNecessary() { private int calculateArchiveCount() { int toArchiveCount = 0; - for (Map.Entry entry : trashData.entrySet()) { - if (!entry.getKey().startsWith("0")) { + for (TrashDataKey key : trashData.keySet()) { + if (key.retryCount != 0) { break; } toArchiveCount++; @@ -203,7 +203,12 @@ public void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type if (context == null) { context = EMPTY_LEDGER_INFO; } - String key = buildKey(RETRY_COUNT, ledgerId, type); + TrashDataKey key = null; + if (ManagedTrash.LEDGER.equals(type)) { + key = TrashDataKey.buildKey(RETRY_COUNT, ledgerId, 0L, type); + } else if (ManagedTrash.OFFLOADED_LEDGER.equals(type)) { + key = TrashDataKey.buildKey(RETRY_COUNT, ledgerId, context.getOffloadContext().getUidMsb(), type); + } trashData.put(key, context); managedTrashMXBean.increaseTotalNumberOfDeleteLedgers(); trashIsDirty = true; @@ -235,9 +240,11 @@ public CompletableFuture asyncUpdateTrashData() { return future; } - private byte[] serialize(Map toPersist) { + + private byte[] serialize(Map toPersist) { + Map transfer = transferTo(toPersist); TrashDataComponent.Builder builder = TrashDataComponent.newBuilder(); - for (Map.Entry entry : toPersist.entrySet()) { + for (Map.Entry entry : transfer.entrySet()) { MLDataFormats.TrashData.Builder innerBuilder = MLDataFormats.TrashData.newBuilder().setKey(entry.getKey()); if (entry.getValue().getLedgerId() != EMPTY_LEDGER_ID) { innerBuilder.setValue(entry.getValue()); @@ -247,10 +254,18 @@ private byte[] serialize(Map toPersist) { return builder.build().toByteArray(); } - private Map deSerialize(byte[] content) throws InvalidProtocolBufferException { + private Map transferTo(Map to) { + Map result = new HashMap<>(); + for (Map.Entry entry : to.entrySet()) { + result.put(entry.getKey().toStringKey(), entry.getValue()); + } + return result; + } + + private Map deSerialize(byte[] content) throws InvalidProtocolBufferException { TrashDataComponent component = TrashDataComponent.parseFrom(content); List componentList = component.getComponentList(); - Map result = new ConcurrentSkipListMap<>(); + Map result = new HashMap<>(); for (MLDataFormats.TrashData ele : componentList) { if (ele.hasValue()) { result.put(ele.getKey(), ele.getValue()); @@ -258,6 +273,15 @@ private Map deSerialize(byte[] content) throws InvalidProtoc result.put(ele.getKey(), EMPTY_LEDGER_INFO); } } + return transferFrom(result); + } + + + private Map transferFrom(Map from) { + Map result = new HashMap<>(); + for (Map.Entry entry : from.entrySet()) { + result.put(TrashDataKey.buildKey(entry.getKey()), entry.getValue()); + } return result; } @@ -275,13 +299,13 @@ private void triggerDelete() { if (!deleteMutex.tryLock()) { return; } - List toDelete = getToDeleteData(); + List toDelete = getToDeleteData(); if (toDelete.size() == 0) { deleteMutex.unlock(); return; } - for (TrashDataKey delHelper : toDelete) { + for (DelHelper delHelper : toDelete) { //unlock in asyncDeleteTrash asyncDeleteTrash(delHelper); } @@ -309,14 +333,14 @@ public CompletableFuture> getAllArchiveIndex() { } @Override - public CompletableFuture> getArchiveData(final long index) { + public CompletableFuture> getArchiveData(final long index) { return metadataStore.get(buildArchivePath(index)).thenComposeAsync(optResult -> { - CompletableFuture> future = new CompletableFuture<>(); + CompletableFuture> future = new CompletableFuture<>(); if (optResult.isPresent()) { byte[] content = optResult.get().getValue(); try { - Map result = deSerialize(content); - future.complete(result); + Map result = deSerialize(content); + future.complete(result.values().stream().toList()); } catch (InvalidProtocolBufferException e) { future.completeExceptionally(e); } @@ -384,7 +408,7 @@ private String buildArchivePath(long index) { } //take 1/10 trash to delete, if the size over 10, use 10 to delete. - private List getToDeleteData() { + private List getToDeleteData() { if (trashData.size() == 0) { return Collections.emptyList(); } @@ -395,14 +419,13 @@ private List getToDeleteData() { if (batchSize == 0) { batchSize = 1; } - List toDelete = new ArrayList<>(batchSize); - for (Map.Entry entry : trashData.descendingMap().entrySet()) { - TrashDataKey delHelper = TrashDataKey.build(entry.getKey(), entry.getValue()); + List toDelete = new ArrayList<>(batchSize); + for (Map.Entry entry : trashData.descendingMap().entrySet()) { //if last retryCount is zero, the before data retryCount is zero too. - if (delHelper.retryCount == 0) { + if (entry.getKey().retryCount == 0) { break; } - toDelete.add(delHelper); + toDelete.add(DelHelper.buildHelper(entry.getKey(), entry.getValue())); if (toDelete.size() == batchSize) { break; } @@ -420,11 +443,11 @@ private void asyncUpdateArchiveData(CompletableFuture future) { return; } //transaction operation - NavigableMap persistDelete = new ConcurrentSkipListMap<>(); - NavigableMap persistArchive = new ConcurrentSkipListMap<>(); + NavigableMap persistDelete = new ConcurrentSkipListMap<>(); + NavigableMap persistArchive = new ConcurrentSkipListMap<>(); - for (Map.Entry entry : trashData.entrySet()) { + for (Map.Entry entry : trashData.entrySet()) { persistArchive.put(entry.getKey(), entry.getValue()); if (persistArchive.size() >= archiveDataLimitSize) { break; @@ -432,7 +455,7 @@ private void asyncUpdateArchiveData(CompletableFuture future) { } persistDelete.putAll(trashData); - for (Map.Entry entry : persistArchive.entrySet()) { + for (Map.Entry entry : persistArchive.entrySet()) { persistDelete.remove(entry.getKey()); } //build delete persist operation @@ -441,10 +464,10 @@ private void asyncUpdateArchiveData(CompletableFuture future) { deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion()), EnumSet.noneOf(CreateOption.class)); //build archive persist operation - Map.Entry lastEntry = persistArchive.lastEntry(); + Map.Entry lastEntry = persistArchive.lastEntry(); OpPut opArchivePersist = - new OpPut(buildArchivePath(TrashDataKey.build(lastEntry.getKey()).ledgerId), - serialize(persistArchive), Optional.of(-1L), EnumSet.noneOf(CreateOption.class)); + new OpPut(buildArchivePath(lastEntry.getKey().ledgerId), serialize(persistArchive), Optional.of(-1L), + EnumSet.noneOf(CreateOption.class)); txOps.add(opDeletePersist); txOps.add(opArchivePersist); metadataStore.batchOperation(txOps); @@ -471,14 +494,9 @@ private void asyncUpdateArchiveData(CompletableFuture future) { } - private static String buildKey(int retryCount, long ledgerId, String suffix) { - return retryCount + TRASH_KEY_SEPARATOR + String.format("%019d", ledgerId) + TRASH_KEY_SEPARATOR - + suffix; - } - - private void asyncDeleteTrash(TrashDataKey delHelper) { - if (delHelper.isLedger()) { - asyncDeleteLedger(delHelper.ledgerId, new AsyncCallbacks.DeleteLedgerCallback() { + private void asyncDeleteTrash(DelHelper delHelper) { + if (delHelper.key.isLedger()) { + asyncDeleteLedger(delHelper.key.ledgerId, new AsyncCallbacks.DeleteLedgerCallback() { @Override public void deleteLedgerComplete(Object ctx) { onDeleteSuccess(delHelper); @@ -489,8 +507,8 @@ public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { onDeleteFailed(delHelper); } }); - } else if (delHelper.isOffloadLedger()) { - asyncDeleteOffloadedLedger(delHelper.ledgerId, delHelper.context, + } else if (delHelper.key.isOffloadLedger()) { + asyncDeleteOffloadedLedger(delHelper.key.ledgerId, delHelper.context, new AsyncCallbacks.DeleteLedgerCallback() { @Override public void deleteLedgerComplete(Object ctx) { @@ -505,19 +523,18 @@ public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { } } - private void onDeleteSuccess(TrashDataKey delHelper) { + private void onDeleteSuccess(DelHelper helper) { try { - String key = delHelper.transferToTrashKey(); if (log.isDebugEnabled()) { String info = null; - if (delHelper.isLedger()) { - info = String.format("[%s] Delete ledger %s success.", name(), delHelper.ledgerId); - } else if (delHelper.isOffloadLedger()) { - info = String.format("[%s] Delete offload ledger %s success.", name(), delHelper.ledgerId); + if (helper.key.isLedger()) { + info = String.format("[%s] Delete ledger %s success.", name(), helper.key.ledgerId); + } else if (helper.key.isOffloadLedger()) { + info = String.format("[%s] Delete offload ledger %s success.", name(), helper.key.ledgerId); } log.debug(info); } - trashData.remove(key); + trashData.remove(helper.key); trashIsDirty = true; } finally { boolean continueToDelete = continueToDelete(); @@ -528,23 +545,22 @@ private void onDeleteSuccess(TrashDataKey delHelper) { } } - private void onDeleteFailed(TrashDataKey delHelper) { + private void onDeleteFailed(DelHelper helper) { try { //override old key - String key = delHelper.transferToTrashKey(); - trashData.remove(key); - trashData.put(buildKey(delHelper.retryCount - 1, delHelper.ledgerId, delHelper.suffix), - delHelper.context); + trashData.remove(helper.key); + trashData.put(TrashDataKey.buildKey(helper.key.retryCount - 1, helper.key.ledgerId, helper.key.msb, + helper.key.suffix), helper.context); trashIsDirty = true; - if (delHelper.retryCount - 1 == 0) { + if (helper.key.retryCount - 1 == 0) { if (log.isWarnEnabled()) { String info = null; - if (delHelper.isLedger()) { - info = String.format("[%s] Delete ledger %d reach retry limit %d.", name(), delHelper.ledgerId, + if (helper.key.isLedger()) { + info = String.format("[%s] Delete ledger %d reach retry limit %d.", name(), helper.key.ledgerId, RETRY_COUNT); - } else if (delHelper.isOffloadLedger()) { + } else if (helper.key.isOffloadLedger()) { info = String.format("[%s] Delete offload ledger %d reach retry limit %d.", name(), - delHelper.ledgerId, RETRY_COUNT); + helper.key.ledgerId, RETRY_COUNT); } log.warn(info); } @@ -560,8 +576,7 @@ private void onDeleteFailed(TrashDataKey delHelper) { } private boolean continueToDelete() { - TrashDataKey delHelper = TrashDataKey.build(trashData.lastEntry().getKey()); - return delHelper.retryCount > 0; + return trashData.lastEntry().getKey().retryCount > 0; } private void asyncDeleteLedger(long ledgerId, AsyncCallbacks.DeleteLedgerCallback callback) { @@ -623,40 +638,54 @@ private static ManagedLedgerException.MetaStoreException getMetaStoreException(T } } + private static class DelHelper { + private final TrashDataKey key; + private final LedgerInfo context; - private static class TrashDataKey { + public DelHelper(TrashDataKey key, LedgerInfo context) { + this.key = key; + this.context = context; + } + + public static DelHelper buildHelper(TrashDataKey key, LedgerInfo context) { + return new DelHelper(key, context); + } + } + + + private static class TrashDataKey implements Comparable { private final int retryCount; private final long ledgerId; - private final String suffix; + private final long msb; - private final LedgerInfo context; + private final String suffix; - public TrashDataKey(int retryCount, long ledgerId, String suffix, LedgerInfo context) { + public TrashDataKey(int retryCount, long ledgerId, long msb, String suffix) { this.retryCount = retryCount; this.ledgerId = ledgerId; + this.msb = msb; this.suffix = suffix; - this.context = context; } - public static TrashDataKey build(String key) { - String[] split = key.split(TRASH_KEY_SEPARATOR); - int retryCont = Integer.parseInt(split[0]); - long ledgerId = Long.parseLong(split[1]); - return new TrashDataKey(retryCont, ledgerId, split[2], null); + private String toStringKey() { + return retryCount + TRASH_KEY_SEPARATOR + ledgerId + TRASH_KEY_SEPARATOR + msb + TRASH_KEY_SEPARATOR + + suffix; } - public static TrashDataKey build(String key, LedgerInfo context) { - String[] split = key.split(TRASH_KEY_SEPARATOR); - int retryCont = Integer.parseInt(split[0]); - long ledgerId = Long.parseLong(split[1]); - return new TrashDataKey(retryCont, ledgerId, split[2], context); + public static TrashDataKey buildKey(int retryCount, long ledgerId, long msb, String suffix) { + return new TrashDataKey(retryCount, ledgerId, msb, suffix); } - private String transferToTrashKey() { - return ManagedTrashImpl.buildKey(retryCount, ledgerId, suffix); + public static TrashDataKey buildKey(String strKey) { + String[] split = strKey.split(TRASH_KEY_SEPARATOR); + int retryCount = Integer.parseInt(split[0]); + long ledgerId = Long.parseLong(split[1]); + long msb = Long.parseLong(split[2]); + String suffix = split[3]; + return new TrashDataKey(retryCount, ledgerId, msb, suffix); } private boolean isLedger() { @@ -666,6 +695,23 @@ private boolean isLedger() { private boolean isOffloadLedger() { return OFFLOADED_LEDGER.equals(suffix); } + + @Override + public int compareTo(TrashDataKey other) { + int c1 = this.retryCount - other.retryCount; + if (c1 != 0) { + return c1; + } + long c2 = this.ledgerId - other.ledgerId; + if (c2 != 0) { + return c2 > 0 ? 1 : -1; + } + long c3 = this.msb - other.msb; + if (c3 != 0) { + return c3 > 0 ? 1 : -1; + } + return this.suffix.compareTo(other.suffix); + } } public enum State { From 26e55a1433e90d3ea680e77b0cab03668d9bc1c5 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Thu, 2 Jun 2022 09:54:35 +0800 Subject: [PATCH 25/43] change TrashDataKey -> TrashKey. --- .../mledger/impl/ManagedTrashImpl.java | 60 +++++++++---------- .../mledger/impl/ManagedTrashTest.java | 49 +++++++++++++++ 2 files changed, 79 insertions(+), 30 deletions(-) create mode 100644 managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index ee059b3757ae7..b7e8e1e1e74a6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -88,7 +88,7 @@ public class ManagedTrashImpl implements ManagedTrash { protected volatile ManagedTrashImpl.State state = null; - private NavigableMap trashData = new ConcurrentSkipListMap<>(); + private NavigableMap trashData = new ConcurrentSkipListMap<>(); //todo 未达到 archiveLimit 的 trashData 中 leftRetryCount == 0 的数据是否需要单独一个节点维护数据 @@ -183,7 +183,7 @@ private void persistTrashIfNecessary() { private int calculateArchiveCount() { int toArchiveCount = 0; - for (TrashDataKey key : trashData.keySet()) { + for (TrashKey key : trashData.keySet()) { if (key.retryCount != 0) { break; } @@ -203,11 +203,11 @@ public void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type if (context == null) { context = EMPTY_LEDGER_INFO; } - TrashDataKey key = null; + TrashKey key = null; if (ManagedTrash.LEDGER.equals(type)) { - key = TrashDataKey.buildKey(RETRY_COUNT, ledgerId, 0L, type); + key = TrashKey.buildKey(RETRY_COUNT, ledgerId, 0L, type); } else if (ManagedTrash.OFFLOADED_LEDGER.equals(type)) { - key = TrashDataKey.buildKey(RETRY_COUNT, ledgerId, context.getOffloadContext().getUidMsb(), type); + key = TrashKey.buildKey(RETRY_COUNT, ledgerId, context.getOffloadContext().getUidMsb(), type); } trashData.put(key, context); managedTrashMXBean.increaseTotalNumberOfDeleteLedgers(); @@ -241,7 +241,7 @@ public CompletableFuture asyncUpdateTrashData() { } - private byte[] serialize(Map toPersist) { + private byte[] serialize(Map toPersist) { Map transfer = transferTo(toPersist); TrashDataComponent.Builder builder = TrashDataComponent.newBuilder(); for (Map.Entry entry : transfer.entrySet()) { @@ -254,15 +254,15 @@ private byte[] serialize(Map toPersist) { return builder.build().toByteArray(); } - private Map transferTo(Map to) { + private Map transferTo(Map to) { Map result = new HashMap<>(); - for (Map.Entry entry : to.entrySet()) { + for (Map.Entry entry : to.entrySet()) { result.put(entry.getKey().toStringKey(), entry.getValue()); } return result; } - private Map deSerialize(byte[] content) throws InvalidProtocolBufferException { + private Map deSerialize(byte[] content) throws InvalidProtocolBufferException { TrashDataComponent component = TrashDataComponent.parseFrom(content); List componentList = component.getComponentList(); Map result = new HashMap<>(); @@ -277,10 +277,10 @@ private Map deSerialize(byte[] content) throws Invalid } - private Map transferFrom(Map from) { - Map result = new HashMap<>(); + private Map transferFrom(Map from) { + Map result = new HashMap<>(); for (Map.Entry entry : from.entrySet()) { - result.put(TrashDataKey.buildKey(entry.getKey()), entry.getValue()); + result.put(TrashKey.buildKey(entry.getKey()), entry.getValue()); } return result; } @@ -339,7 +339,7 @@ public CompletableFuture> getArchiveData(final long index) { if (optResult.isPresent()) { byte[] content = optResult.get().getValue(); try { - Map result = deSerialize(content); + Map result = deSerialize(content); future.complete(result.values().stream().toList()); } catch (InvalidProtocolBufferException e) { future.completeExceptionally(e); @@ -420,7 +420,7 @@ private List getToDeleteData() { batchSize = 1; } List toDelete = new ArrayList<>(batchSize); - for (Map.Entry entry : trashData.descendingMap().entrySet()) { + for (Map.Entry entry : trashData.descendingMap().entrySet()) { //if last retryCount is zero, the before data retryCount is zero too. if (entry.getKey().retryCount == 0) { break; @@ -443,11 +443,11 @@ private void asyncUpdateArchiveData(CompletableFuture future) { return; } //transaction operation - NavigableMap persistDelete = new ConcurrentSkipListMap<>(); - NavigableMap persistArchive = new ConcurrentSkipListMap<>(); + NavigableMap persistDelete = new ConcurrentSkipListMap<>(); + NavigableMap persistArchive = new ConcurrentSkipListMap<>(); - for (Map.Entry entry : trashData.entrySet()) { + for (Map.Entry entry : trashData.entrySet()) { persistArchive.put(entry.getKey(), entry.getValue()); if (persistArchive.size() >= archiveDataLimitSize) { break; @@ -455,7 +455,7 @@ private void asyncUpdateArchiveData(CompletableFuture future) { } persistDelete.putAll(trashData); - for (Map.Entry entry : persistArchive.entrySet()) { + for (Map.Entry entry : persistArchive.entrySet()) { persistDelete.remove(entry.getKey()); } //build delete persist operation @@ -464,7 +464,7 @@ private void asyncUpdateArchiveData(CompletableFuture future) { deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion()), EnumSet.noneOf(CreateOption.class)); //build archive persist operation - Map.Entry lastEntry = persistArchive.lastEntry(); + Map.Entry lastEntry = persistArchive.lastEntry(); OpPut opArchivePersist = new OpPut(buildArchivePath(lastEntry.getKey().ledgerId), serialize(persistArchive), Optional.of(-1L), EnumSet.noneOf(CreateOption.class)); @@ -549,7 +549,7 @@ private void onDeleteFailed(DelHelper helper) { try { //override old key trashData.remove(helper.key); - trashData.put(TrashDataKey.buildKey(helper.key.retryCount - 1, helper.key.ledgerId, helper.key.msb, + trashData.put(TrashKey.buildKey(helper.key.retryCount - 1, helper.key.ledgerId, helper.key.msb, helper.key.suffix), helper.context); trashIsDirty = true; if (helper.key.retryCount - 1 == 0) { @@ -639,21 +639,21 @@ private static ManagedLedgerException.MetaStoreException getMetaStoreException(T } private static class DelHelper { - private final TrashDataKey key; + private final TrashKey key; private final LedgerInfo context; - public DelHelper(TrashDataKey key, LedgerInfo context) { + public DelHelper(TrashKey key, LedgerInfo context) { this.key = key; this.context = context; } - public static DelHelper buildHelper(TrashDataKey key, LedgerInfo context) { + public static DelHelper buildHelper(TrashKey key, LedgerInfo context) { return new DelHelper(key, context); } } - private static class TrashDataKey implements Comparable { + private static class TrashKey implements Comparable { private final int retryCount; @@ -663,7 +663,7 @@ private static class TrashDataKey implements Comparable { private final String suffix; - public TrashDataKey(int retryCount, long ledgerId, long msb, String suffix) { + public TrashKey(int retryCount, long ledgerId, long msb, String suffix) { this.retryCount = retryCount; this.ledgerId = ledgerId; this.msb = msb; @@ -675,17 +675,17 @@ private String toStringKey() { + suffix; } - public static TrashDataKey buildKey(int retryCount, long ledgerId, long msb, String suffix) { - return new TrashDataKey(retryCount, ledgerId, msb, suffix); + public static TrashKey buildKey(int retryCount, long ledgerId, long msb, String suffix) { + return new TrashKey(retryCount, ledgerId, msb, suffix); } - public static TrashDataKey buildKey(String strKey) { + public static TrashKey buildKey(String strKey) { String[] split = strKey.split(TRASH_KEY_SEPARATOR); int retryCount = Integer.parseInt(split[0]); long ledgerId = Long.parseLong(split[1]); long msb = Long.parseLong(split[2]); String suffix = split[3]; - return new TrashDataKey(retryCount, ledgerId, msb, suffix); + return new TrashKey(retryCount, ledgerId, msb, suffix); } private boolean isLedger() { @@ -697,7 +697,7 @@ private boolean isOffloadLedger() { } @Override - public int compareTo(TrashDataKey other) { + public int compareTo(TrashKey other) { int c1 = this.retryCount - other.retryCount; if (c1 != 0) { return c1; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java new file mode 100644 index 0000000000000..d15e148155211 --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import static org.testng.Assert.assertEquals; +import com.google.common.base.Charsets; +import java.nio.charset.Charset; +import java.util.List; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.Test; + +public class ManagedTrashTest extends MockedBookKeeperTestCase { + + private static final Charset Encoding = Charsets.UTF_8; + + private static final Logger log = LoggerFactory.getLogger(ManagedTrashTest.class); + + + @Test(timeOut = 20000) + void readFromEmptyLedger() throws Exception { + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + ledger.addEntry("test".getBytes(Encoding)); + } +} From 5a2c4a40db4c6fcb69a03c15438458d36b93976b Mon Sep 17 00:00:00 2001 From: horizonzy Date: Thu, 2 Jun 2022 12:53:44 +0800 Subject: [PATCH 26/43] add backoff policy to avoid same key to delete frequently in a short time. --- .../mledger/impl/ManagedTrashImpl.java | 56 ++++++++++--------- 1 file changed, 31 insertions(+), 25 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index b7e8e1e1e74a6..d9b374d37d67b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -77,10 +77,12 @@ public class ManagedTrashImpl implements ManagedTrash { private static final String TRASH_KEY_SEPARATOR = "-"; - private static final int RETRY_COUNT = 9; + private static final int RETRY_COUNT = 10; private static final long EMPTY_LEDGER_ID = -1L; + private static final long DELETION_INTERNAL_MILLIS = TimeUnit.MINUTES.toMillis(1); + private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().setLedgerId(EMPTY_LEDGER_ID).build(); private static final AtomicReferenceFieldUpdater STATE_UPDATER = @@ -300,11 +302,12 @@ private void triggerDelete() { return; } List toDelete = getToDeleteData(); - if (toDelete.size() == 0) { deleteMutex.unlock(); return; } + toDelete.removeIf(ele -> System.currentTimeMillis() - ele.key.lastDeleteTs < DELETION_INTERNAL_MILLIS); + for (DelHelper delHelper : toDelete) { //unlock in asyncDeleteTrash asyncDeleteTrash(delHelper); @@ -537,11 +540,8 @@ private void onDeleteSuccess(DelHelper helper) { trashData.remove(helper.key); trashIsDirty = true; } finally { - boolean continueToDelete = continueToDelete(); deleteMutex.unlock(); - if (continueToDelete) { - triggerDeleteInBackground(); - } + continueDeleteIfNecessary(); } } @@ -549,8 +549,11 @@ private void onDeleteFailed(DelHelper helper) { try { //override old key trashData.remove(helper.key); - trashData.put(TrashKey.buildKey(helper.key.retryCount - 1, helper.key.ledgerId, helper.key.msb, - helper.key.suffix), helper.context); + + TrashKey trashKey = TrashKey.buildKey(helper.key.retryCount - 1, helper.key.ledgerId, helper.key.msb, + helper.key.type); + trashKey.lastDeleteTs = System.currentTimeMillis(); + trashData.put(trashKey, helper.context); trashIsDirty = true; if (helper.key.retryCount - 1 == 0) { if (log.isWarnEnabled()) { @@ -567,16 +570,16 @@ private void onDeleteFailed(DelHelper helper) { increaseArchiveCountWhenDeleteFailed(); } } finally { - boolean continueToDelete = continueToDelete(); deleteMutex.unlock(); - if (continueToDelete) { - triggerDeleteInBackground(); - } + continueDeleteIfNecessary(); } } - private boolean continueToDelete() { - return trashData.lastEntry().getKey().retryCount > 0; + private void continueDeleteIfNecessary() { + if (trashData.lastEntry().getKey().retryCount > 0) { + scheduledExecutor.schedule(this::triggerDeleteInBackground, DELETION_INTERNAL_MILLIS / 5, + TimeUnit.MILLISECONDS); + } } private void asyncDeleteLedger(long ledgerId, AsyncCallbacks.DeleteLedgerCallback callback) { @@ -659,24 +662,27 @@ private static class TrashKey implements Comparable { private final long ledgerId; + //the same ledgerId maybe correspond two offload storage. private final long msb; - private final String suffix; + private final String type; + + private long lastDeleteTs; - public TrashKey(int retryCount, long ledgerId, long msb, String suffix) { + public TrashKey(int retryCount, long ledgerId, long msb, String type) { this.retryCount = retryCount; this.ledgerId = ledgerId; this.msb = msb; - this.suffix = suffix; + this.type = type; } private String toStringKey() { return retryCount + TRASH_KEY_SEPARATOR + ledgerId + TRASH_KEY_SEPARATOR + msb + TRASH_KEY_SEPARATOR - + suffix; + + type; } - public static TrashKey buildKey(int retryCount, long ledgerId, long msb, String suffix) { - return new TrashKey(retryCount, ledgerId, msb, suffix); + public static TrashKey buildKey(int retryCount, long ledgerId, long msb, String type) { + return new TrashKey(retryCount, ledgerId, msb, type); } public static TrashKey buildKey(String strKey) { @@ -684,16 +690,16 @@ public static TrashKey buildKey(String strKey) { int retryCount = Integer.parseInt(split[0]); long ledgerId = Long.parseLong(split[1]); long msb = Long.parseLong(split[2]); - String suffix = split[3]; - return new TrashKey(retryCount, ledgerId, msb, suffix); + String type = split[3]; + return new TrashKey(retryCount, ledgerId, msb, type); } private boolean isLedger() { - return LEDGER.equals(suffix); + return LEDGER.equals(type); } private boolean isOffloadLedger() { - return OFFLOADED_LEDGER.equals(suffix); + return OFFLOADED_LEDGER.equals(type); } @Override @@ -710,7 +716,7 @@ public int compareTo(TrashKey other) { if (c3 != 0) { return c3 > 0 ? 1 : -1; } - return this.suffix.compareTo(other.suffix); + return this.type.compareTo(other.type); } } From 59a21fc0e4e503ff764e2ba52bcd3813eb6434ae Mon Sep 17 00:00:00 2001 From: horizonzy Date: Thu, 2 Jun 2022 13:18:10 +0800 Subject: [PATCH 27/43] code clean. --- .../bookkeeper/mledger/impl/ManagedTrashImpl.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index d9b374d37d67b..23cb4e1252de7 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -550,10 +550,10 @@ private void onDeleteFailed(DelHelper helper) { //override old key trashData.remove(helper.key); - TrashKey trashKey = TrashKey.buildKey(helper.key.retryCount - 1, helper.key.ledgerId, helper.key.msb, + TrashKey newKey = TrashKey.buildKey(helper.key.retryCount - 1, helper.key.ledgerId, helper.key.msb, helper.key.type); - trashKey.lastDeleteTs = System.currentTimeMillis(); - trashData.put(trashKey, helper.context); + newKey.markLastDeleteTs(); + trashData.put(newKey, helper.context); trashIsDirty = true; if (helper.key.retryCount - 1 == 0) { if (log.isWarnEnabled()) { @@ -676,6 +676,10 @@ public TrashKey(int retryCount, long ledgerId, long msb, String type) { this.type = type; } + private void markLastDeleteTs() { + this.lastDeleteTs = System.currentTimeMillis(); + } + private String toStringKey() { return retryCount + TRASH_KEY_SEPARATOR + ledgerId + TRASH_KEY_SEPARATOR + msb + TRASH_KEY_SEPARATOR + type; From c121e423f0198e46055b0793b2b3b0df23b6d6df Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 6 Jun 2022 14:58:37 +0800 Subject: [PATCH 28/43] complete unit test. --- .../mledger/ManagedLedgerConfig.java | 19 +- .../bookkeeper/mledger/ManagedTrash.java | 26 +- .../mledger/impl/ManagedLedgerImpl.java | 13 +- .../mledger/impl/ManagedTrashDisableImpl.java | 6 +- .../mledger/impl/ManagedTrashImpl.java | 338 ++++++++++-------- .../mledger/impl/ManagedTrashTest.java | 327 ++++++++++++++++- .../pulsar/broker/ServiceConfiguration.java | 17 +- .../pulsar/broker/service/BrokerService.java | 2 + .../metadata/impl/AbstractMetadataStore.java | 2 +- .../metadata/impl/EtcdMetadataStore.java | 2 +- .../impl/LocalMemoryMetadataStore.java | 8 +- .../pulsar/metadata/impl/ZKMetadataStore.java | 2 +- .../AbstractBatchedMetadataStore.java | 2 +- .../client/PulsarMockBookKeeper.java | 6 +- 14 files changed, 572 insertions(+), 198 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 2878c55383c8d..f73d10e096b1b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -81,8 +81,9 @@ public class ManagedLedgerConfig { @Getter @Setter private boolean cacheEvictionByMarkDeletedPosition = false; - private int archiveDataLimitSize = 500; private boolean supportTwoPhaseDeletion = false; + private int archiveDataLimitSize = 500; + private int deleteIntervalSeconds = 60000; public boolean isCreateIfMissing() { return createIfMissing; @@ -685,6 +686,14 @@ public void setInactiveLedgerRollOverTime(int inactiveLedgerRollOverTimeMs, Time this.inactiveLedgerRollOverTimeMs = (int) unit.toMillis(inactiveLedgerRollOverTimeMs); } + public boolean isSupportTwoPhaseDeletion() { + return supportTwoPhaseDeletion; + } + + public void setSupportTwoPhaseDeletion(boolean supportTwoPhaseDeletion) { + this.supportTwoPhaseDeletion = supportTwoPhaseDeletion; + } + public int getArchiveDataLimitSize() { return archiveDataLimitSize; } @@ -693,11 +702,11 @@ public void setArchiveDataLimitSize(int archiveDataLimitSize) { this.archiveDataLimitSize = archiveDataLimitSize; } - public boolean isSupportTwoPhaseDeletion() { - return supportTwoPhaseDeletion; + public int getDeleteIntervalSeconds() { + return deleteIntervalSeconds; } - public void setSupportTwoPhaseDeletion(boolean supportTwoPhaseDeletion) { - this.supportTwoPhaseDeletion = supportTwoPhaseDeletion; + public void setDeleteIntervalSeconds(int deleteIntervalSeconds) { + this.deleteIntervalSeconds = deleteIntervalSeconds; } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 16a92274941e9..a5a50b177e7ae 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -19,28 +19,36 @@ package org.apache.bookkeeper.mledger; import java.util.List; -import java.util.Map; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; public interface ManagedTrash { - String MANAGED_LEDGER = "managed-ledger"; + enum ManagedType { + MANAGED_LEDGER("managed-ledger"), + MANAGED_CURSOR("managed-cursor"), + SCHEMA("schema"); + private final String name; - String MANAGED_CURSOR = "managed-cursor"; + ManagedType(String name) { + this.name = name; + } - String SCHEMA = "schema"; + public String getName() { + return name; + } + } - String LEDGER = "L"; - - String OFFLOADED_LEDGER = "OL"; + enum LedgerType { + OFFLOAD_LEDGER, + LEDGER + } String name(); CompletableFuture initialize(); - void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) - throws ManagedLedgerException; + void appendLedgerTrashData(long ledgerId, LedgerInfo context, LedgerType type) throws ManagedLedgerException; CompletableFuture asyncUpdateTrashData(); 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 ffd72882d1e16..0e412c6f0d1e1 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 @@ -341,7 +341,7 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper } this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); this.managedTrash = config.isSupportTwoPhaseDeletion() - ? new ManagedTrashImpl(ManagedTrash.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, + ? new ManagedTrashImpl(ManagedTrash.ManagedType.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, executor, bookKeeper) : new ManagedTrashDisableImpl(); } @@ -2682,11 +2682,10 @@ private CompletableFuture asyncUpdateTrashData(Collection deletableLedg CompletableFuture future = new CompletableFuture<>(); try { for (Long ledgerId : deletableLedgerIds) { - managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.LEDGER); + managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.LedgerType.LEDGER); } for (Long ledgerId : deletableOffloadedLedgerIds) { - managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), - ManagedTrash.OFFLOADED_LEDGER); + managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), ManagedTrash.LedgerType.OFFLOAD_LEDGER); } future.complete(null); } catch (ManagedLedgerException e) { @@ -3081,7 +3080,7 @@ private void offloadLoop(CompletableFuture promise, Queue prepareLedgerInfoForOffloaded(long ledgerId, UUI "Previous failed offload"); } else { managedTrash.appendLedgerTrashData(ledgerId, oldInfo, - ManagedTrash.OFFLOADED_LEDGER); + ManagedTrash.LedgerType.OFFLOAD_LEDGER); managedTrash.asyncUpdateTrashData(); } } @@ -3908,7 +3907,7 @@ protected boolean checkAndCompleteLedgerOpTask(int rc, LedgerHandle lh, Object c asyncDeleteLedger(lh.getId(), DEFAULT_LEDGER_DELETE_RETRIES); } else { try { - managedTrash.appendLedgerTrashData(lh.getId(), null, ManagedTrash.LEDGER); + managedTrash.appendLedgerTrashData(lh.getId(), null, ManagedTrash.LedgerType.LEDGER); managedTrash.asyncUpdateTrashData(); } catch (ManagedLedgerException e) { log.warn("[{}]-{} Failed to append trash data.", this.name, lh.getId()); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index 50798105fb1b5..f2159339f7212 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -19,11 +19,10 @@ package org.apache.bookkeeper.mledger.impl; import java.util.List; -import java.util.Map; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedTrash; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; public class ManagedTrashDisableImpl implements ManagedTrash { @@ -41,7 +40,8 @@ public CompletableFuture initialize() { } @Override - public void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) { + public void appendLedgerTrashData(long ledgerId, LedgerInfo context, LedgerType type) + throws ManagedLedgerException { } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 23cb4e1252de7..501488c927329 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -24,8 +24,6 @@ import com.google.protobuf.InvalidProtocolBufferException; import java.util.ArrayList; import java.util.Collections; -import java.util.EnumSet; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -51,13 +49,10 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.TrashDataComponent; import org.apache.bookkeeper.mledger.util.CallbackMutex; import org.apache.commons.collections4.CollectionUtils; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.Stat; -import org.apache.pulsar.metadata.api.extended.CreateOption; -import org.apache.pulsar.metadata.impl.batching.AbstractBatchedMetadataStore; -import org.apache.pulsar.metadata.impl.batching.MetadataOp; -import org.apache.pulsar.metadata.impl.batching.OpPut; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,7 +60,7 @@ public class ManagedTrashImpl implements ManagedTrash { private static final Logger log = LoggerFactory.getLogger(ManagedTrashImpl.class); - private static final String BASE_NODE = "/trash-data"; + private static final String BASE_NODE = "/managed-trash"; private static final String PREFIX = BASE_NODE + "/"; @@ -77,12 +72,10 @@ public class ManagedTrashImpl implements ManagedTrash { private static final String TRASH_KEY_SEPARATOR = "-"; - private static final int RETRY_COUNT = 10; + private static final int RETRY_COUNT = 3; private static final long EMPTY_LEDGER_ID = -1L; - private static final long DELETION_INTERNAL_MILLIS = TimeUnit.MINUTES.toMillis(1); - private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().setLedgerId(EMPTY_LEDGER_ID).build(); private static final AtomicReferenceFieldUpdater STATE_UPDATER = @@ -98,9 +91,11 @@ public class ManagedTrashImpl implements ManagedTrash { private final CallbackMutex deleteMutex = new CallbackMutex(); - private final CallbackMutex trashPersistMutex = new CallbackMutex(); + private final CallbackMutex trashMutex = new CallbackMutex(); + + private final CallbackMutex archiveMutex = new CallbackMutex(); - private final AbstractBatchedMetadataStore metadataStore; + private final MetadataStore metadataStore; private volatile Stat deleteStat; @@ -118,27 +113,27 @@ public class ManagedTrashImpl implements ManagedTrash { private final int archiveDataLimitSize; + private final long deleteIntervalMillis; + private volatile boolean trashIsDirty; private ScheduledFuture checkTrashPersistTask; private final ManagedTrashMXBean managedTrashMXBean; - public ManagedTrashImpl(String type, String name, MetadataStore metadataStore, ManagedLedgerConfig config, + public ManagedTrashImpl(ManagedType type, String name, MetadataStore metadataStore, ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, OrderedExecutor executor, BookKeeper bookKeeper) { - this.type = type; + this.type = type.getName(); this.name = name; this.config = config; - if (!(metadataStore instanceof AbstractBatchedMetadataStore)) { - throw new IllegalStateException("ManagedTrashImpl metadata store must support batch operation."); - } - STATE_UPDATER.set(this, State.None); - this.metadataStore = (AbstractBatchedMetadataStore) metadataStore; + this.metadataStore = metadataStore; this.scheduledExecutor = scheduledExecutor; this.executor = executor; this.bookKeeper = bookKeeper; this.archiveDataLimitSize = config.getArchiveDataLimitSize(); + this.deleteIntervalMillis = TimeUnit.SECONDS.toMillis(config.getDeleteIntervalSeconds()); this.managedTrashMXBean = new ManagedTrashMXBeanImpl(this); + STATE_UPDATER.set(this, State.None); } @Override @@ -156,6 +151,10 @@ public CompletableFuture initialize() { return; } if (res.isEmpty()) { + STATE_UPDATER.set(this, State.INITIALIZED); + checkTrashPersistTask = + scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, + TimeUnit.MINUTES); future.complete(null); return; } @@ -195,7 +194,7 @@ private int calculateArchiveCount() { } @Override - public void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type) + public void appendLedgerTrashData(long ledgerId, LedgerInfo context, LedgerType type) throws ManagedLedgerException { State state = STATE_UPDATER.get(this); if (state != State.INITIALIZED) { @@ -206,9 +205,9 @@ public void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type context = EMPTY_LEDGER_INFO; } TrashKey key = null; - if (ManagedTrash.LEDGER.equals(type)) { + if (ManagedTrash.LedgerType.LEDGER.equals(type)) { key = TrashKey.buildKey(RETRY_COUNT, ledgerId, 0L, type); - } else if (ManagedTrash.OFFLOADED_LEDGER.equals(type)) { + } else if (ManagedTrash.LedgerType.OFFLOAD_LEDGER.equals(type)) { key = TrashKey.buildKey(RETRY_COUNT, ledgerId, context.getOffloadContext().getUidMsb(), type); } trashData.put(key, context); @@ -221,29 +220,42 @@ public void appendLedgerTrashData(long ledgerId, LedgerInfo context, String type public CompletableFuture asyncUpdateTrashData() { log.info("{} Start async update trash data", name()); CompletableFuture future = new CompletableFuture<>(); + doAsyncUpdateTrashData(future); + return future; + } + + public void asyncUpdateTrashDataInBackground(CompletableFuture future) { + executor.executeOrdered(name, safeRun(() -> doAsyncUpdateTrashData(future))); + } + + private void doAsyncUpdateTrashData(CompletableFuture future) { State state = STATE_UPDATER.get(this); if (state != State.INITIALIZED) { future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(new IllegalStateException( String.format("[%s] is not initialized, current state: %s", name(), state)))); - return future; + return; + } + if (!trashMutex.tryLock()) { + scheduledExecutor.schedule(() -> asyncUpdateTrashDataInBackground(future), 100, TimeUnit.MILLISECONDS); + return; } metadataStore.put(buildDeletePath(), serialize(trashData), deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion())) .whenCompleteAsync((res, e) -> { if (e != null) { future.completeExceptionally(getMetaStoreException(e)); + trashMutex.unlock(); return; } deleteStat = res; trashIsDirty = false; - trashPersistMutex.unlock(); future.complete(null); + trashMutex.unlock(); }, executor.chooseThread(name)); - return future; } - private byte[] serialize(Map toPersist) { + public byte[] serialize(Map toPersist) { Map transfer = transferTo(toPersist); TrashDataComponent.Builder builder = TrashDataComponent.newBuilder(); for (Map.Entry entry : transfer.entrySet()) { @@ -257,17 +269,17 @@ private byte[] serialize(Map toPersist) { } private Map transferTo(Map to) { - Map result = new HashMap<>(); + Map result = new ConcurrentSkipListMap<>(); for (Map.Entry entry : to.entrySet()) { result.put(entry.getKey().toStringKey(), entry.getValue()); } return result; } - private Map deSerialize(byte[] content) throws InvalidProtocolBufferException { + public NavigableMap deSerialize(byte[] content) throws InvalidProtocolBufferException { TrashDataComponent component = TrashDataComponent.parseFrom(content); List componentList = component.getComponentList(); - Map result = new HashMap<>(); + Map result = new ConcurrentSkipListMap<>(); for (MLDataFormats.TrashData ele : componentList) { if (ele.hasValue()) { result.put(ele.getKey(), ele.getValue()); @@ -279,8 +291,8 @@ private Map deSerialize(byte[] content) throws InvalidProt } - private Map transferFrom(Map from) { - Map result = new HashMap<>(); + private NavigableMap transferFrom(Map from) { + NavigableMap result = new ConcurrentSkipListMap<>(); for (Map.Entry entry : from.entrySet()) { result.put(TrashKey.buildKey(entry.getKey()), entry.getValue()); } @@ -306,12 +318,16 @@ private void triggerDelete() { deleteMutex.unlock(); return; } - toDelete.removeIf(ele -> System.currentTimeMillis() - ele.key.lastDeleteTs < DELETION_INTERNAL_MILLIS); + toDelete.removeIf(ele -> System.currentTimeMillis() - ele.key.lastDeleteTs < deleteIntervalMillis); + List> futures = new ArrayList<>(); for (DelHelper delHelper : toDelete) { - //unlock in asyncDeleteTrash - asyncDeleteTrash(delHelper); + futures.add(asyncDeleteTrash(delHelper)); } + FutureUtil.waitForAll(futures).whenCompleteAsync((res, e) -> { + deleteMutex.unlock(); + continueDeleteIfNecessary(); + }); } @Override @@ -437,6 +453,10 @@ private List getToDeleteData() { } private void asyncUpdateArchiveData(CompletableFuture future) { + if (!archiveMutex.tryLock()) { + future.complete(null); + return; + } log.info("[{}] Start async update archive data", name()); State state = STATE_UPDATER.get(this); @@ -445,164 +465,153 @@ private void asyncUpdateArchiveData(CompletableFuture future) { String.format("[%s] is not initialized, current state: %s", name(), state)))); return; } - //transaction operation - NavigableMap persistDelete = new ConcurrentSkipListMap<>(); - NavigableMap persistArchive = new ConcurrentSkipListMap<>(); - - for (Map.Entry entry : trashData.entrySet()) { - persistArchive.put(entry.getKey(), entry.getValue()); - if (persistArchive.size() >= archiveDataLimitSize) { - break; + asyncUpdateTrashData().thenAccept(ignore -> { + NavigableMap persistArchive = new ConcurrentSkipListMap<>(); + //here we didn't lock trashData, so maybe the persistArchive is discontinuous. such as: 1,2,3,10,12... + for (Map.Entry entry : trashData.entrySet()) { + persistArchive.put(entry.getKey(), entry.getValue()); + if (persistArchive.size() >= archiveDataLimitSize) { + break; + } } - } - persistDelete.putAll(trashData); - for (Map.Entry entry : persistArchive.entrySet()) { - persistDelete.remove(entry.getKey()); - } - //build delete persist operation - List txOps = new ArrayList<>(2); - OpPut opDeletePersist = new OpPut(buildDeletePath(), serialize(persistDelete), - deleteStat == null ? Optional.of(-1L) : Optional.of(deleteStat.getVersion()), - EnumSet.noneOf(CreateOption.class)); - //build archive persist operation - Map.Entry lastEntry = persistArchive.lastEntry(); - OpPut opArchivePersist = - new OpPut(buildArchivePath(lastEntry.getKey().ledgerId), serialize(persistArchive), Optional.of(-1L), - EnumSet.noneOf(CreateOption.class)); - txOps.add(opDeletePersist); - txOps.add(opArchivePersist); - metadataStore.batchOperation(txOps); - - opDeletePersist.getFuture().whenCompleteAsync((res, e) -> { - if (e != null) { - log.error("[{}] Persist trash data failed.", name(), e); - future.completeExceptionally(getMetaStoreException(e)); - return; - } - opArchivePersist.getFuture().whenCompleteAsync((res1, e1) -> { - if (e1 != null) { - log.error("[{}] Persist archive data failed.", name(), e1); - future.completeExceptionally(getMetaStoreException(e1)); + Map.Entry lastEntry = persistArchive.lastEntry(); + + metadataStore.put(buildArchivePath(lastEntry.getKey().ledgerId), serialize(persistArchive), + Optional.of(-1L)).whenCompleteAsync((res, e) -> { + if (e != null) { + log.error("[{}] Persist archive data failed.", name(), e); + future.completeExceptionally(getMetaStoreException(e)); + deleteMutex.unlock(); return; } - deleteStat = res; - trashData = persistDelete; + persistArchive.keySet().forEach(ele -> trashData.remove(ele)); trashIsDirty = false; - toArchiveCount.set(0); - future.complete(null); + for (int i = 0; i < archiveDataLimitSize; i++) { + toArchiveCount.decrementAndGet(); + } + asyncUpdateTrashData().whenComplete((res1, e1) -> { + if (e1 != null) { + future.completeExceptionally(getMetaStoreException(e1)); + archiveMutex.unlock(); + return; + } + future.complete(null); + archiveMutex.unlock(); + }); }, executor.chooseThread(name)); - }, executor.chooseThread(name)); + }).exceptionally(e -> { + log.error("[{}] Persist archive data failed.", name(), e); + future.completeExceptionally(getMetaStoreException(e)); + archiveMutex.unlock(); + return null; + }); + } - private void asyncDeleteTrash(DelHelper delHelper) { + private CompletableFuture asyncDeleteTrash(DelHelper delHelper) { if (delHelper.key.isLedger()) { - asyncDeleteLedger(delHelper.key.ledgerId, new AsyncCallbacks.DeleteLedgerCallback() { - @Override - public void deleteLedgerComplete(Object ctx) { - onDeleteSuccess(delHelper); - } - - @Override - public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { + CompletableFuture future = asyncDeleteLedger(delHelper.key.ledgerId); + future.whenCompleteAsync((res, e) -> { + if (e != null) { onDeleteFailed(delHelper); + return; } - }); + onDeleteSuccess(delHelper); + }, executor.chooseThread(name)); + return future; } else if (delHelper.key.isOffloadLedger()) { - asyncDeleteOffloadedLedger(delHelper.key.ledgerId, delHelper.context, - new AsyncCallbacks.DeleteLedgerCallback() { - @Override - public void deleteLedgerComplete(Object ctx) { - onDeleteSuccess(delHelper); - } - - @Override - public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { - onDeleteFailed(delHelper); - } - }); + CompletableFuture future = asyncDeleteOffloadedLedger(delHelper.key.ledgerId, delHelper.context); + future.whenCompleteAsync((res, e) -> { + if (e != null) { + onDeleteFailed(delHelper); + return; + } + onDeleteSuccess(delHelper); + }, executor.chooseThread(name)); + return future; } + return CompletableFuture.completedFuture(null); } private void onDeleteSuccess(DelHelper helper) { - try { - if (log.isDebugEnabled()) { - String info = null; - if (helper.key.isLedger()) { - info = String.format("[%s] Delete ledger %s success.", name(), helper.key.ledgerId); - } else if (helper.key.isOffloadLedger()) { - info = String.format("[%s] Delete offload ledger %s success.", name(), helper.key.ledgerId); - } - log.debug(info); + if (log.isDebugEnabled()) { + String info = null; + if (helper.key.isLedger()) { + info = String.format("[%s] Delete ledger %s success.", name(), helper.key.ledgerId); + } else if (helper.key.isOffloadLedger()) { + info = String.format("[%s] Delete offload ledger %s success.", name(), helper.key.ledgerId); } - trashData.remove(helper.key); - trashIsDirty = true; - } finally { - deleteMutex.unlock(); - continueDeleteIfNecessary(); + log.debug(info); } + trashData.remove(helper.key); + trashIsDirty = true; } private void onDeleteFailed(DelHelper helper) { - try { - //override old key - trashData.remove(helper.key); - - TrashKey newKey = TrashKey.buildKey(helper.key.retryCount - 1, helper.key.ledgerId, helper.key.msb, - helper.key.type); - newKey.markLastDeleteTs(); - trashData.put(newKey, helper.context); - trashIsDirty = true; - if (helper.key.retryCount - 1 == 0) { - if (log.isWarnEnabled()) { - String info = null; - if (helper.key.isLedger()) { - info = String.format("[%s] Delete ledger %d reach retry limit %d.", name(), helper.key.ledgerId, - RETRY_COUNT); - } else if (helper.key.isOffloadLedger()) { - info = String.format("[%s] Delete offload ledger %d reach retry limit %d.", name(), - helper.key.ledgerId, RETRY_COUNT); - } - log.warn(info); + //override old key + trashData.remove(helper.key); + + TrashKey newKey = TrashKey.buildKey(helper.key.retryCount - 1, helper.key.ledgerId, helper.key.msb, + helper.key.type); + newKey.markLastDeleteTs(); + trashData.put(newKey, helper.context); + trashIsDirty = true; + if (helper.key.retryCount - 1 == 0) { + if (log.isWarnEnabled()) { + String info = null; + if (helper.key.isLedger()) { + info = String.format("[%s] Delete ledger %d reach retry limit %d.", name(), helper.key.ledgerId, + RETRY_COUNT); + } else if (helper.key.isOffloadLedger()) { + info = String.format("[%s] Delete offload ledger %d reach retry limit %d.", name(), + helper.key.ledgerId, RETRY_COUNT); } - increaseArchiveCountWhenDeleteFailed(); + log.warn(info); } - } finally { - deleteMutex.unlock(); - continueDeleteIfNecessary(); + increaseArchiveCountWhenDeleteFailed(); } } private void continueDeleteIfNecessary() { - if (trashData.lastEntry().getKey().retryCount > 0) { - scheduledExecutor.schedule(this::triggerDeleteInBackground, DELETION_INTERNAL_MILLIS / 5, + Map.Entry lastEntry = trashData.lastEntry(); + if (lastEntry == null) { + return; + } + if (lastEntry.getKey().retryCount > 0) { + scheduledExecutor.schedule(this::triggerDeleteInBackground, deleteIntervalMillis / 5, TimeUnit.MILLISECONDS); } } - private void asyncDeleteLedger(long ledgerId, AsyncCallbacks.DeleteLedgerCallback callback) { + private CompletableFuture asyncDeleteLedger(long ledgerId) { + CompletableFuture future = new CompletableFuture<>(); log.info("[{}] Start async delete ledger {}", name(), ledgerId); bookKeeper.asyncDeleteLedger(ledgerId, (rc, ctx) -> { if (isNoSuchLedgerExistsException(rc)) { log.warn("[{}] Ledger was already deleted {}", name(), ledgerId); } else if (rc != BKException.Code.OK) { log.error("[{}] Error delete ledger {} : {}", name(), ledgerId, BKException.getMessage(rc)); - callback.deleteLedgerFailed(ManagedLedgerImpl.createManagedLedgerException(rc), null); + future.completeExceptionally(ManagedLedgerImpl.createManagedLedgerException(rc)); return; } if (log.isDebugEnabled()) { log.debug("[{}] Deleted ledger {}", name(), ledgerId); } - callback.deleteLedgerComplete(ctx); + future.complete(null); }, null); + return future; } - private void asyncDeleteOffloadedLedger(long ledgerId, LedgerInfo info, - AsyncCallbacks.DeleteLedgerCallback callback) { + private CompletableFuture asyncDeleteOffloadedLedger(long ledgerId, LedgerInfo info) { + CompletableFuture future = new CompletableFuture<>(); if (!info.getOffloadContext().hasUidMsb()) { - return; + future.completeExceptionally(new IllegalArgumentException( + String.format("[%s] Failed delete offload for ledgerId %s, can't find offload context.", name(), + ledgerId))); + return future; } String cleanupReason = "Trash-Trimming"; @@ -622,15 +631,16 @@ private void asyncDeleteOffloadedLedger(long ledgerId, LedgerInfo info, if (exception != null) { log.warn("[{}] Failed delete offload for ledgerId {} uuid {}, (cleanup reason: {})", name(), ledgerId, uuid, cleanupReason, exception); - callback.deleteLedgerFailed(new ManagedLedgerException("Failed to delete offloaded ledger"), - null); + future.completeExceptionally( + new ManagedLedgerException("Failed to delete offloaded ledger")); return; } - callback.deleteLedgerComplete(null); + future.complete(null); }); } catch (Exception e) { log.warn("[{}] Failed to delete offloaded ledgers.", name(), e); } + return future; } private static ManagedLedgerException.MetaStoreException getMetaStoreException(Throwable t) { @@ -655,8 +665,7 @@ public static DelHelper buildHelper(TrashKey key, LedgerInfo context) { } } - - private static class TrashKey implements Comparable { + public static class TrashKey implements Comparable { private final int retryCount; @@ -665,11 +674,11 @@ private static class TrashKey implements Comparable { //the same ledgerId maybe correspond two offload storage. private final long msb; - private final String type; + private final LedgerType type; private long lastDeleteTs; - public TrashKey(int retryCount, long ledgerId, long msb, String type) { + public TrashKey(int retryCount, long ledgerId, long msb, LedgerType type) { this.retryCount = retryCount; this.ledgerId = ledgerId; this.msb = msb; @@ -685,7 +694,7 @@ private String toStringKey() { + type; } - public static TrashKey buildKey(int retryCount, long ledgerId, long msb, String type) { + public static TrashKey buildKey(int retryCount, long ledgerId, long msb, LedgerType type) { return new TrashKey(retryCount, ledgerId, msb, type); } @@ -694,20 +703,39 @@ public static TrashKey buildKey(String strKey) { int retryCount = Integer.parseInt(split[0]); long ledgerId = Long.parseLong(split[1]); long msb = Long.parseLong(split[2]); - String type = split[3]; + LedgerType type = LedgerType.valueOf(split[3]); return new TrashKey(retryCount, ledgerId, msb, type); } + public int getRetryCount() { + return retryCount; + } + + public long getLedgerId() { + return ledgerId; + } + + public long getMsb() { + return msb; + } + + public LedgerType getType() { + return type; + } + private boolean isLedger() { - return LEDGER.equals(type); + return LedgerType.LEDGER.equals(type); } private boolean isOffloadLedger() { - return OFFLOADED_LEDGER.equals(type); + return LedgerType.OFFLOAD_LEDGER.equals(type); } @Override public int compareTo(TrashKey other) { + if (other == this) { + return 0; + } int c1 = this.retryCount - other.retryCount; if (c1 != 0) { return c1; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java index d15e148155211..303d7064f6627 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java @@ -20,30 +20,341 @@ import static org.testng.Assert.assertEquals; import com.google.common.base.Charsets; +import java.lang.reflect.Field; import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.HashMap; import java.util.List; -import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ManagedCursor; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.bookkeeper.client.AsyncCallback; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.PulsarMockBookKeeper; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedTrash; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.Notification; +import org.apache.pulsar.metadata.api.NotificationType; +import org.apache.pulsar.metadata.api.Stat; +import org.apache.pulsar.metadata.api.extended.CreateOption; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.impl.FaultInjectionMetadataStore; +import org.apache.pulsar.metadata.impl.LocalMemoryMetadataStore; +import org.awaitility.Awaitility; +import org.testng.Assert; import org.testng.annotations.Test; public class ManagedTrashTest extends MockedBookKeeperTestCase { private static final Charset Encoding = Charsets.UTF_8; - private static final Logger log = LoggerFactory.getLogger(ManagedTrashTest.class); + private List deletedLedgers = new ArrayList<>(); + PulsarMockBookKeeper bkc; + + FaultInjectionMetadataStore metadataStore; + + private Map persistedData = new HashMap<>(); + + + @Override + protected void setUpTestCase() throws Exception { + MetadataStoreExtended metadataStoreExtended = + new LocalMemoryMetadataStore("memory:local", MetadataStoreConfig.builder().build()) { + @Override + public CompletableFuture storePut(String path, byte[] data, Optional optExpectedVersion, + EnumSet options) { + if (!isValidPath(path)) { + return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path)); + } + synchronized (map) { + boolean hasVersion = optExpectedVersion.isPresent(); + int expectedVersion = optExpectedVersion.orElse(-1L).intValue(); + + if (options.contains(CreateOption.Sequential)) { + path += Long.toString(sequentialIdGenerator.getAndIncrement()); + } + + long now = System.currentTimeMillis(); + + if (hasVersion && expectedVersion == -1) { + Value newValue = new Value(0, data, now, now, options.contains(CreateOption.Ephemeral)); + Value existingValue = map.putIfAbsent(path, newValue); + if (existingValue != null) { + return FutureUtils.exception(new MetadataStoreException.BadVersionException("")); + } else { + persistedData.put(path, data); + receivedNotification(new Notification(NotificationType.Created, path)); + notifyParentChildrenChanged(path); + return FutureUtils.value(new Stat(path, 0, now, now, newValue.isEphemeral(), true)); + } + } else { + Value existingValue = map.get(path); + long existingVersion = existingValue != null ? existingValue.getVersion() : -1; + if (hasVersion && expectedVersion != existingVersion) { + return FutureUtils.exception(new MetadataStoreException.BadVersionException("")); + } else { + long newVersion = existingValue != null ? existingValue.getVersion() + 1 : 0; + long createdTimestamp = + existingValue != null ? existingValue.getCreatedTimestamp() : now; + Value newValue = new Value(newVersion, data, createdTimestamp, now, + options.contains(CreateOption.Ephemeral)); + persistedData.put(path, data); + map.put(path, newValue); + + NotificationType type = + existingValue == null ? NotificationType.Created : + NotificationType.Modified; + receivedNotification(new Notification(type, path)); + if (type == NotificationType.Created) { + notifyParentChildrenChanged(path); + } + return FutureUtils + .value(new Stat(path, newValue.getVersion(), newValue.getCreatedTimestamp(), + newValue.getModifiedTimestamp(), + false, true)); + } + } + } + } + }; + metadataStore = new FaultInjectionMetadataStore(metadataStoreExtended); + bkc = new PulsarMockBookKeeper(executor) { + @Override + public void asyncDeleteLedger(long lId, AsyncCallback.DeleteCallback cb, Object ctx) { + getProgrammedFailure().thenComposeAsync((res) -> { + if (lId >= 10000) { + throw new IllegalArgumentException("LedgerId is invalid"); + } + if (ledgers.containsKey(lId)) { + ledgers.remove(lId); + deletedLedgers.add(lId); + return FutureUtils.value(null); + } else { + return FutureUtils.exception(new BKException.BKNoSuchLedgerExistsException()); + } + }, executor).whenCompleteAsync((res, exception) -> { + if (exception != null) { + cb.deleteComplete(getExceptionCode(exception), ctx); + } else { + cb.deleteComplete(BKException.Code.OK, ctx); + } + }, executor); + } + }; + factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); + } + + @Override + protected void cleanUpTestCase() throws Exception { + super.cleanUpTestCase(); + deletedLedgers.clear(); + persistedData.clear(); + } + + @Test + public void testTrashKeyOrder() throws Exception { + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setArchiveDataLimitSize(10); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + managedTrash.appendLedgerTrashData(10, null, ManagedTrash.LedgerType.LEDGER); + managedTrash.appendLedgerTrashData(6, null, ManagedTrash.LedgerType.LEDGER); + managedTrash.appendLedgerTrashData(100, null, ManagedTrash.LedgerType.LEDGER); + managedTrash.appendLedgerTrashData(3, null, ManagedTrash.LedgerType.LEDGER); + managedTrash.appendLedgerTrashData(7, null, ManagedTrash.LedgerType.LEDGER); + + + LedgerInfo.Builder builder = LedgerInfo.newBuilder().setLedgerId(7); + UUID uuid = UUID.randomUUID(); + builder.getOffloadContextBuilder() + .setUidMsb(uuid.getMostSignificantBits()) + .setUidLsb(uuid.getLeastSignificantBits()); + LedgerInfo ledgerInfo = builder.build(); + managedTrash.appendLedgerTrashData(7, ledgerInfo, ManagedTrash.LedgerType.OFFLOAD_LEDGER); + + Field field1 = ManagedTrashImpl.class.getDeclaredField("trashData"); + field1.setAccessible(true); + ConcurrentSkipListMap map = (ConcurrentSkipListMap) field1.get(managedTrash); + + Map.Entry entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 3); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 6); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 7); + Assert.assertEquals(entry.getKey().getMsb(), uuid.getMostSignificantBits()); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.OFFLOAD_LEDGER); + + entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 7); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 10); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 100); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + } + + @Test + public void testManagedTrashDelete() throws Exception { + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setMaxEntriesPerLedger(10); + managedLedgerConfig.setRetentionTime(1, TimeUnit.SECONDS); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + for (int i = 0; i < 100; i++) { + ledger.addEntry("test".getBytes(Encoding)); + } + Awaitility.await().untilAsserted(() -> { + assertEquals(deletedLedgers.size(), 9); + }); + } + + @Test + public void testManagedTrashArchive() throws Exception { + int ledgerCount = 30; + int archiveLimit = 10; - @Test(timeOut = 20000) - void readFromEmptyLedger() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); - ledger.addEntry("test".getBytes(Encoding)); + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + + //the ledgerId >= 10000, it will delete failed. see line_142. + for (int i = 0; i < ledgerCount; i++) { + managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); + } + managedTrash.triggerDeleteInBackground(); + + Awaitility.await().untilAsserted(() -> { + assertEquals(5, persistedData.size()); + }); + + assertEquals(managedTrash.getTrashDataSize(), 0); + + Field field2 = ManagedTrashImpl.class.getDeclaredField("toArchiveCount"); + field2.setAccessible(true); + AtomicInteger toArchiveCount = (AtomicInteger) field2.get(managedTrash); + + assertEquals(toArchiveCount.get(), 0); + + + ConcurrentSkipListMap totalArchive = new ConcurrentSkipListMap<>(); + for (Map.Entry entry : persistedData.entrySet()) { + if (entry.getKey().startsWith("/managed-trash")) { + if (entry.getKey().endsWith("/delete")) { + byte[] value = entry.getValue(); + assertEquals(value.length, 0); + } else { + Map archiveData = + managedTrash.deSerialize(entry.getValue()); + assertEquals(archiveData.size(), archiveLimit); + totalArchive.putAll(archiveData); + } + } + } + + assertEquals(totalArchive.size(), ledgerCount); + int index = 0; + for (Map.Entry entry : totalArchive.entrySet()) { + assertEquals(entry.getKey().getLedgerId(), 10000 + index); + index++; + } + + } + + @Test + public void testManagedTrashClose() throws Exception { + //when managedTrash close, it will persist trashData. + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setArchiveDataLimitSize(10); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + managedTrash.appendLedgerTrashData(10, null, ManagedTrash.LedgerType.LEDGER); + managedTrash.appendLedgerTrashData(6, null, ManagedTrash.LedgerType.LEDGER); + managedTrash.appendLedgerTrashData(100, null, ManagedTrash.LedgerType.LEDGER); + managedTrash.appendLedgerTrashData(3, null, ManagedTrash.LedgerType.LEDGER); + managedTrash.appendLedgerTrashData(7, null, ManagedTrash.LedgerType.LEDGER); + ledger.close(); + assertEquals(persistedData.size(), 2); + // //managed_ledger/my_test_ledger/delete + byte[] content = persistedData.get("/managed-trash/managed-ledger/my_test_ledger/delete"); + NavigableMap persistedTrashData = + managedTrash.deSerialize(content); + + assertEquals(persistedTrashData.size(), 5); + + Map.Entry entry = + persistedTrashData.pollFirstEntry(); + + Assert.assertEquals(entry.getKey().getLedgerId(), 3); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + entry = persistedTrashData.pollFirstEntry(); + + Assert.assertEquals(entry.getKey().getLedgerId(), 6); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + entry = persistedTrashData.pollFirstEntry(); + + Assert.assertEquals(entry.getKey().getLedgerId(), 7); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + entry = persistedTrashData.pollFirstEntry(); + + Assert.assertEquals(entry.getKey().getLedgerId(), 10); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + entry = persistedTrashData.pollFirstEntry(); + + Assert.assertEquals(entry.getKey().getLedgerId(), 100); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index dc3252097b414..25568da9ad8ae 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2517,10 +2517,25 @@ public class ServiceConfiguration implements PulsarConfiguration { @FieldContext( dynamic = true, category = CATEGORY_STORAGE_ML, - doc = "Using two phase deletion when delete ledger. (Default value is false)" + doc = "Using two phase deletion when delete ledger. if true, " + + "managedTrash will take over ledger deletion. (Default false)" ) private boolean managedLedgerSupportTwoPhaseDeletion; + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "To control managedTrash archive data size, if reach, persist archive data. (Default 500)" + ) + private int managedTrashArchiveDataLimitSize = 500; + + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "The interval of deletion at same ledger in managedTrash. (Default 60 seconds)" + ) + private int managedTrashDeleteInternalSeconds = 60; + @FieldContext( category = CATEGORY_STORAGE_ML, doc = "Evicting cache data by the slowest markDeletedPosition or readPosition. " diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 74608a8c6a4b5..9c99118506eef 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1579,6 +1579,8 @@ public CompletableFuture getManagedLedgerConfig(TopicName t serviceConfig.isCacheEvictionByMarkDeletedPosition()); managedLedgerConfig.setSupportTwoPhaseDeletion( serviceConfig.isManagedLedgerSupportTwoPhaseDeletion()); + managedLedgerConfig.setArchiveDataLimitSize(serviceConfig.getManagedTrashArchiveDataLimitSize()); + managedLedgerConfig.setDeleteIntervalSeconds(serviceConfig.getManagedTrashDeleteInternalSeconds()); OffloadPoliciesImpl nsLevelOffloadPolicies = (OffloadPoliciesImpl) policies.map(p -> p.offload_policies).orElse(null); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java index cac386405a64c..c173395e2b701 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java @@ -340,7 +340,7 @@ protected static String parent(String path) { * 2. starts with '/' * 3. not ends with '/', except root path "/" */ - static boolean isValidPath(String path) { + protected static boolean isValidPath(String path) { return StringUtils.equals(path, "/") || StringUtils.isNotBlank(path) && path.startsWith("/") diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java index 707410cb2ccae..e1c26bead096a 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java @@ -159,7 +159,7 @@ protected CompletableFuture storePut(String path, byte[] data, Optional ops) { + protected void batchOperation(List ops) { try { Txn txn = kv.txn(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java index 63efba8f7246d..da53c60947b07 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java @@ -32,6 +32,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import lombok.Data; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.pulsar.common.util.FutureUtil; @@ -52,7 +53,8 @@ public class LocalMemoryMetadataStore extends AbstractMetadataStore implements M static final String MEMORY_SCHEME_IDENTIFIER = "memory:"; @Data - private static class Value { + @Getter + protected static class Value { final long version; final byte[] data; final long createdTimestamp; @@ -60,8 +62,8 @@ private static class Value { final boolean ephemeral; } - private final NavigableMap map; - private final AtomicLong sequentialIdGenerator; + protected final NavigableMap map; + protected final AtomicLong sequentialIdGenerator; private static final Map> STATIC_MAPS = new MapMaker() .weakValues().makeMap(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index badb45f189d3f..6697934d56b3a 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -149,7 +149,7 @@ protected void receivedSessionEvent(SessionEvent event) { } @Override - public void batchOperation(List ops) { + protected void batchOperation(List ops) { try { zkc.multi(ops.stream().map(this::convertOp).collect(Collectors.toList()), (rc, path, ctx, results) -> { if (results == null) { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java index dd17cd8f5512b..616cac289efea 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java @@ -158,5 +158,5 @@ private void enqueue(MessagePassingQueue queue, MetadataOp op) { } } - public abstract void batchOperation(List ops); + protected abstract void batchOperation(List ops); } diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java index 50df4b6b0df0f..802e924326cd0 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -75,7 +75,7 @@ public ClientConfiguration getConf() { return super.getConf(); } - final Map ledgers = new ConcurrentHashMap<>(); + protected final Map ledgers = new ConcurrentHashMap<>(); final AtomicLong sequence = new AtomicLong(3); CompletableFuture defaultResponse = CompletableFuture.completedFuture(null); @@ -317,7 +317,7 @@ synchronized boolean checkReturnEmptyLedger() { return shouldFailNow; } - synchronized CompletableFuture getProgrammedFailure() { + protected synchronized CompletableFuture getProgrammedFailure() { return failures.isEmpty() ? defaultResponse : failures.remove(0); } @@ -351,7 +351,7 @@ public synchronized void addEntryDelay(long delay, TimeUnit unit) { addEntryDelaysMillis.add(unit.toMillis(delay)); } - static int getExceptionCode(Throwable t) { + protected static int getExceptionCode(Throwable t) { if (t instanceof BKException) { return ((BKException) t).getCode(); } else if (t.getCause() != null) { From 9dd87e2d97ac02717e771dcb9387f85c34725e96 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 6 Jun 2022 15:09:22 +0800 Subject: [PATCH 29/43] code clean --- .../mledger/ManagedLedgerConfig.java | 2 +- .../mledger/impl/ManagedLedgerImpl.java | 7 +- .../mledger/impl/ManagedTrashImpl.java | 14 ++-- .../mledger/impl/ManagedTrashTest.java | 79 +++---------------- .../impl/LocalMemoryMetadataStore.java | 8 +- 5 files changed, 28 insertions(+), 82 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index f73d10e096b1b..264cb34f54414 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -83,7 +83,7 @@ public class ManagedLedgerConfig { private boolean cacheEvictionByMarkDeletedPosition = false; private boolean supportTwoPhaseDeletion = false; private int archiveDataLimitSize = 500; - private int deleteIntervalSeconds = 60000; + private int deleteIntervalSeconds = 60; public boolean isCreateIfMissing() { return createIfMissing; 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 80804051a0af5..511189409658e 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 @@ -341,8 +341,8 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper } this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); this.managedTrash = config.isSupportTwoPhaseDeletion() - ? new ManagedTrashImpl(ManagedTrash.ManagedType.MANAGED_LEDGER, name, metadataStore, config, scheduledExecutor, - executor, bookKeeper) : new ManagedTrashDisableImpl(); + ? new ManagedTrashImpl(ManagedTrash.ManagedType.MANAGED_LEDGER, name, metadataStore, config, + scheduledExecutor, executor, bookKeeper) : new ManagedTrashDisableImpl(); } synchronized void initialize(final ManagedLedgerInitializeLedgerCallback callback, final Object ctx) { @@ -2688,7 +2688,8 @@ private CompletableFuture asyncUpdateTrashData(Collection deletableLedg managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.LedgerType.LEDGER); } for (Long ledgerId : deletableOffloadedLedgerIds) { - managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), ManagedTrash.LedgerType.OFFLOAD_LEDGER); + managedTrash.appendLedgerTrashData(ledgerId, ledgers.get(ledgerId), + ManagedTrash.LedgerType.OFFLOAD_LEDGER); } future.complete(null); } catch (ManagedLedgerException e) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 501488c927329..c983726d05ff6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -151,7 +151,7 @@ public CompletableFuture initialize() { return; } if (res.isEmpty()) { - STATE_UPDATER.set(this, State.INITIALIZED); + STATE_UPDATER.set(this, State.Initialized); checkTrashPersistTask = scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, TimeUnit.MINUTES); @@ -167,7 +167,7 @@ public CompletableFuture initialize() { checkTrashPersistTask = scheduledExecutor.scheduleAtFixedRate(safeRun(this::persistTrashIfNecessary), 30L, 30L, TimeUnit.MINUTES); - STATE_UPDATER.set(this, State.INITIALIZED); + STATE_UPDATER.set(this, State.Initialized); triggerDeleteInBackground(); } catch (InvalidProtocolBufferException exc) { future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(exc)); @@ -197,7 +197,7 @@ private int calculateArchiveCount() { public void appendLedgerTrashData(long ledgerId, LedgerInfo context, LedgerType type) throws ManagedLedgerException { State state = STATE_UPDATER.get(this); - if (state != State.INITIALIZED) { + if (state != State.Initialized) { throw ManagedLedgerException.getManagedLedgerException(new IllegalStateException( String.format("[%s] is not initialized, current state: %s", name(), state))); } @@ -230,7 +230,7 @@ public void asyncUpdateTrashDataInBackground(CompletableFuture future) { private void doAsyncUpdateTrashData(CompletableFuture future) { State state = STATE_UPDATER.get(this); - if (state != State.INITIALIZED) { + if (state != State.Initialized) { future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(new IllegalStateException( String.format("[%s] is not initialized, current state: %s", name(), state)))); return; @@ -306,7 +306,7 @@ public void triggerDeleteInBackground() { private void triggerDelete() { State state = STATE_UPDATER.get(this); - if (state != State.INITIALIZED) { + if (state != State.Initialized) { log.warn("[{}] is not initialized, current state: {}", name(), state); return; } @@ -460,7 +460,7 @@ private void asyncUpdateArchiveData(CompletableFuture future) { log.info("[{}] Start async update archive data", name()); State state = STATE_UPDATER.get(this); - if (state != State.INITIALIZED) { + if (state != State.Initialized) { future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(new IllegalStateException( String.format("[%s] is not initialized, current state: %s", name(), state)))); return; @@ -754,7 +754,7 @@ public int compareTo(TrashKey other) { public enum State { None, - INITIALIZED, + Initialized, Closed, } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java index 303d7064f6627..0e9bb7a082af3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java @@ -23,7 +23,6 @@ import java.lang.reflect.Field; import java.nio.charset.Charset; import java.util.ArrayList; -import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -43,16 +42,10 @@ import org.apache.bookkeeper.mledger.ManagedTrash; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; -import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataStoreConfig; -import org.apache.pulsar.metadata.api.MetadataStoreException; -import org.apache.pulsar.metadata.api.Notification; -import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.Stat; -import org.apache.pulsar.metadata.api.extended.CreateOption; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.apache.pulsar.metadata.impl.FaultInjectionMetadataStore; -import org.apache.pulsar.metadata.impl.LocalMemoryMetadataStore; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.Test; @@ -72,66 +65,20 @@ public class ManagedTrashTest extends MockedBookKeeperTestCase { @Override protected void setUpTestCase() throws Exception { - MetadataStoreExtended metadataStoreExtended = - new LocalMemoryMetadataStore("memory:local", MetadataStoreConfig.builder().build()) { - @Override - public CompletableFuture storePut(String path, byte[] data, Optional optExpectedVersion, - EnumSet options) { - if (!isValidPath(path)) { - return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path)); - } - synchronized (map) { - boolean hasVersion = optExpectedVersion.isPresent(); - int expectedVersion = optExpectedVersion.orElse(-1L).intValue(); - - if (options.contains(CreateOption.Sequential)) { - path += Long.toString(sequentialIdGenerator.getAndIncrement()); - } - - long now = System.currentTimeMillis(); - - if (hasVersion && expectedVersion == -1) { - Value newValue = new Value(0, data, now, now, options.contains(CreateOption.Ephemeral)); - Value existingValue = map.putIfAbsent(path, newValue); - if (existingValue != null) { - return FutureUtils.exception(new MetadataStoreException.BadVersionException("")); - } else { - persistedData.put(path, data); - receivedNotification(new Notification(NotificationType.Created, path)); - notifyParentChildrenChanged(path); - return FutureUtils.value(new Stat(path, 0, now, now, newValue.isEphemeral(), true)); - } - } else { - Value existingValue = map.get(path); - long existingVersion = existingValue != null ? existingValue.getVersion() : -1; - if (hasVersion && expectedVersion != existingVersion) { - return FutureUtils.exception(new MetadataStoreException.BadVersionException("")); - } else { - long newVersion = existingValue != null ? existingValue.getVersion() + 1 : 0; - long createdTimestamp = - existingValue != null ? existingValue.getCreatedTimestamp() : now; - Value newValue = new Value(newVersion, data, createdTimestamp, now, - options.contains(CreateOption.Ephemeral)); - persistedData.put(path, data); - map.put(path, newValue); - - NotificationType type = - existingValue == null ? NotificationType.Created : - NotificationType.Modified; - receivedNotification(new Notification(type, path)); - if (type == NotificationType.Created) { - notifyParentChildrenChanged(path); - } - return FutureUtils - .value(new Stat(path, newValue.getVersion(), newValue.getCreatedTimestamp(), - newValue.getModifiedTimestamp(), - false, true)); - } - } - } + metadataStore = new FaultInjectionMetadataStore( + MetadataStoreExtended.create("memory:local", MetadataStoreConfig.builder().build())) { + @Override + public CompletableFuture put(String path, byte[] value, Optional expectedVersion) { + CompletableFuture future = super.put(path, value, expectedVersion); + future.whenComplete((res, e) -> { + if (e != null) { + return; } - }; - metadataStore = new FaultInjectionMetadataStore(metadataStoreExtended); + persistedData.put(path, value); + }); + return future; + } + }; bkc = new PulsarMockBookKeeper(executor) { @Override public void asyncDeleteLedger(long lId, AsyncCallback.DeleteCallback cb, Object ctx) { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java index da53c60947b07..63efba8f7246d 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java @@ -32,7 +32,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import lombok.Data; -import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.pulsar.common.util.FutureUtil; @@ -53,8 +52,7 @@ public class LocalMemoryMetadataStore extends AbstractMetadataStore implements M static final String MEMORY_SCHEME_IDENTIFIER = "memory:"; @Data - @Getter - protected static class Value { + private static class Value { final long version; final byte[] data; final long createdTimestamp; @@ -62,8 +60,8 @@ protected static class Value { final boolean ephemeral; } - protected final NavigableMap map; - protected final AtomicLong sequentialIdGenerator; + private final NavigableMap map; + private final AtomicLong sequentialIdGenerator; private static final Map> STATIC_MAPS = new MapMaker() .weakValues().makeMap(); From 82dd67c62d64b0835e2bbd3707c86125af5015e0 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 6 Jun 2022 17:57:09 +0800 Subject: [PATCH 30/43] check retryCount is 0 when persist archive data. --- .../apache/bookkeeper/mledger/impl/ManagedTrashImpl.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index c983726d05ff6..cb9f9badbc469 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -465,15 +465,15 @@ private void asyncUpdateArchiveData(CompletableFuture future) { String.format("[%s] is not initialized, current state: %s", name(), state)))); return; } - asyncUpdateTrashData().thenAccept(ignore -> { NavigableMap persistArchive = new ConcurrentSkipListMap<>(); //here we didn't lock trashData, so maybe the persistArchive is discontinuous. such as: 1,2,3,10,12... for (Map.Entry entry : trashData.entrySet()) { - persistArchive.put(entry.getKey(), entry.getValue()); - if (persistArchive.size() >= archiveDataLimitSize) { + //in theory, the retryCount can't more than 0. + if (entry.getKey().retryCount > 0 || persistArchive.size() >= archiveDataLimitSize) { break; } + persistArchive.put(entry.getKey(), entry.getValue()); } Map.Entry lastEntry = persistArchive.lastEntry(); @@ -488,7 +488,7 @@ private void asyncUpdateArchiveData(CompletableFuture future) { } persistArchive.keySet().forEach(ele -> trashData.remove(ele)); trashIsDirty = false; - for (int i = 0; i < archiveDataLimitSize; i++) { + for (int i = 0; i < persistArchive.size(); i++) { toArchiveCount.decrementAndGet(); } asyncUpdateTrashData().whenComplete((res1, e1) -> { From 9db6dec364edeba968ea5e9222839fec4de52d2f Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 6 Jun 2022 23:21:51 +0800 Subject: [PATCH 31/43] use timestamp to take place of ledgerId as archive node suffix. --- .../org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index cb9f9badbc469..d08b53898be40 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -476,9 +476,7 @@ private void asyncUpdateArchiveData(CompletableFuture future) { persistArchive.put(entry.getKey(), entry.getValue()); } - Map.Entry lastEntry = persistArchive.lastEntry(); - - metadataStore.put(buildArchivePath(lastEntry.getKey().ledgerId), serialize(persistArchive), + metadataStore.put(buildArchivePath(System.currentTimeMillis()), serialize(persistArchive), Optional.of(-1L)).whenCompleteAsync((res, e) -> { if (e != null) { log.error("[{}] Persist archive data failed.", name(), e); From 896720e0efc06c8ff6c517858306265f12f0de9b Mon Sep 17 00:00:00 2001 From: horizonzy Date: Wed, 8 Jun 2022 09:32:55 +0800 Subject: [PATCH 32/43] add test unit to cover code. --- .../bookkeeper/mledger/ManagedTrash.java | 4 +- .../mledger/impl/ManagedTrashDisableImpl.java | 5 +- .../mledger/impl/ManagedTrashImpl.java | 8 +- .../mledger/impl/ManagedTrashTest.java | 84 ++++++++++++++++++- 4 files changed, 90 insertions(+), 11 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index a5a50b177e7ae..20c5678f2846d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -19,7 +19,9 @@ package org.apache.bookkeeper.mledger; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.impl.ManagedTrashImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; public interface ManagedTrash { @@ -56,7 +58,7 @@ enum LedgerType { CompletableFuture> getAllArchiveIndex(); - CompletableFuture> getArchiveData(long index); + CompletableFuture> getArchiveData(long index); long getTrashDataSize(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index f2159339f7212..5613a35274168 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -19,6 +19,7 @@ package org.apache.bookkeeper.mledger.impl; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedLedgerException; @@ -59,8 +60,8 @@ public CompletableFuture> getAllArchiveIndex() { } @Override - public CompletableFuture> getArchiveData(long index) { - return (CompletableFuture>) COMPLETABLE_FUTURE; + public CompletableFuture> getArchiveData(long index) { + return (CompletableFuture>) COMPLETABLE_FUTURE; } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index d08b53898be40..e2d0e5c4ec42b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -352,14 +352,14 @@ public CompletableFuture> getAllArchiveIndex() { } @Override - public CompletableFuture> getArchiveData(final long index) { + public CompletableFuture> getArchiveData(final long index) { return metadataStore.get(buildArchivePath(index)).thenComposeAsync(optResult -> { - CompletableFuture> future = new CompletableFuture<>(); + CompletableFuture> future = new CompletableFuture<>(); if (optResult.isPresent()) { byte[] content = optResult.get().getValue(); try { Map result = deSerialize(content); - future.complete(result.values().stream().toList()); + future.complete(result); } catch (InvalidProtocolBufferException e) { future.completeExceptionally(e); } @@ -469,7 +469,7 @@ private void asyncUpdateArchiveData(CompletableFuture future) { NavigableMap persistArchive = new ConcurrentSkipListMap<>(); //here we didn't lock trashData, so maybe the persistArchive is discontinuous. such as: 1,2,3,10,12... for (Map.Entry entry : trashData.entrySet()) { - //in theory, the retryCount can't more than 0. + //in theory, the retryCount can't greater than 0. if (entry.getKey().retryCount > 0 || persistArchive.size() >= archiveDataLimitSize) { break; } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java index 0e9bb7a082af3..eb3691a2f78aa 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java @@ -190,7 +190,7 @@ public void testManagedTrashDelete() throws Exception { @Test public void testManagedTrashArchive() throws Exception { - int ledgerCount = 30; + int entryCount = 30; int archiveLimit = 10; ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); @@ -204,7 +204,7 @@ public void testManagedTrashArchive() throws Exception { //the ledgerId >= 10000, it will delete failed. see line_142. - for (int i = 0; i < ledgerCount; i++) { + for (int i = 0; i < entryCount; i++) { managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); } managedTrash.triggerDeleteInBackground(); @@ -237,7 +237,7 @@ public void testManagedTrashArchive() throws Exception { } } - assertEquals(totalArchive.size(), ledgerCount); + assertEquals(totalArchive.size(), entryCount); int index = 0; for (Map.Entry entry : totalArchive.entrySet()) { assertEquals(entry.getKey().getLedgerId(), 10000 + index); @@ -265,7 +265,7 @@ public void testManagedTrashClose() throws Exception { managedTrash.appendLedgerTrashData(7, null, ManagedTrash.LedgerType.LEDGER); ledger.close(); assertEquals(persistedData.size(), 2); - // //managed_ledger/my_test_ledger/delete + byte[] content = persistedData.get("/managed-trash/managed-ledger/my_test_ledger/delete"); NavigableMap persistedTrashData = managedTrash.deSerialize(content); @@ -302,6 +302,82 @@ public void testManagedTrashClose() throws Exception { Assert.assertEquals(entry.getKey().getLedgerId(), 100); Assert.assertEquals(entry.getKey().getMsb(), 0); Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + } + + @Test + public void testGetAllArchiveIndex() throws Exception { + int entryCount = 30; + int archiveLimit = 10; + + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + //the ledgerId >= 10000, it will delete failed. see line_142. + for (int i = 0; i < entryCount; i++) { + managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); + } + managedTrash.triggerDeleteInBackground(); + + Awaitility.await().untilAsserted(() -> { + assertEquals(5, persistedData.size()); + }); + + CompletableFuture> index = managedTrash.getAllArchiveIndex(); + + assertEquals(index.get().size(), 3); } + + @Test + public void testGetArchiveData() throws Exception { + int entryCount = 30; + int archiveLimit = 10; + + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + + //the ledgerId >= 10000, it will delete failed. see line_142. + for (int i = 0; i < entryCount; i++) { + managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); + } + managedTrash.triggerDeleteInBackground(); + + Awaitility.await().untilAsserted(() -> { + assertEquals(5, persistedData.size()); + }); + + CompletableFuture> indexFuture = managedTrash.getAllArchiveIndex(); + + List indexes = indexFuture.get(); + + Map trashedData = new ConcurrentSkipListMap<>(); + for (Long index : indexes) { + trashedData.putAll(managedTrash.getArchiveData(index).get()); + } + assertEquals(trashedData.size(), 30); + + int i = 0; + for (Map.Entry entry : trashedData.entrySet()) { + Assert.assertEquals(entry.getKey().getRetryCount(), 0); + Assert.assertEquals(entry.getKey().getLedgerId(), 10000 + i); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + i++; + } + + } + } From 655251444594f4f2a327fd44e917aaa28774a776 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Wed, 8 Jun 2022 18:48:27 +0800 Subject: [PATCH 33/43] when didn't get lock to delete, make next delete procedure no delay. --- .../bookkeeper/mledger/impl/ManagedTrashImpl.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index e2d0e5c4ec42b..d784c35eeb417 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -99,6 +99,8 @@ public class ManagedTrashImpl implements ManagedTrash { private volatile Stat deleteStat; + private volatile boolean continueDeleteImmediately; + private final String type; private final String name; @@ -311,6 +313,7 @@ private void triggerDelete() { return; } if (!deleteMutex.tryLock()) { + continueDeleteImmediately = true; return; } List toDelete = getToDeleteData(); @@ -579,8 +582,13 @@ private void continueDeleteIfNecessary() { return; } if (lastEntry.getKey().retryCount > 0) { - scheduledExecutor.schedule(this::triggerDeleteInBackground, deleteIntervalMillis / 5, - TimeUnit.MILLISECONDS); + if (continueDeleteImmediately) { + triggerDeleteInBackground(); + continueDeleteImmediately = false; + } else { + scheduledExecutor.schedule(this::triggerDeleteInBackground, deleteIntervalMillis / 5, + TimeUnit.MILLISECONDS); + } } } From 6cfa99de7f11da70f520c09d359cdf4da1176a64 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Wed, 8 Jun 2022 18:52:11 +0800 Subject: [PATCH 34/43] when didn't get lock to delete, make next delete procedure no delay. --- .../apache/bookkeeper/mledger/impl/ManagedTrashImpl.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index d784c35eeb417..ca273850a3e4a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -99,7 +99,7 @@ public class ManagedTrashImpl implements ManagedTrash { private volatile Stat deleteStat; - private volatile boolean continueDeleteImmediately; + private final AtomicInteger continueDeleteImmediately = new AtomicInteger(); private final String type; @@ -313,7 +313,7 @@ private void triggerDelete() { return; } if (!deleteMutex.tryLock()) { - continueDeleteImmediately = true; + continueDeleteImmediately.incrementAndGet(); return; } List toDelete = getToDeleteData(); @@ -582,9 +582,9 @@ private void continueDeleteIfNecessary() { return; } if (lastEntry.getKey().retryCount > 0) { - if (continueDeleteImmediately) { + if (continueDeleteImmediately.get() > 0) { triggerDeleteInBackground(); - continueDeleteImmediately = false; + continueDeleteImmediately.decrementAndGet(); } else { scheduledExecutor.schedule(this::triggerDeleteInBackground, deleteIntervalMillis / 5, TimeUnit.MILLISECONDS); From 0b21f49a11cd8f4f3e1e090fb78eb9d5f72d588d Mon Sep 17 00:00:00 2001 From: horizonzy Date: Wed, 8 Jun 2022 20:23:52 +0800 Subject: [PATCH 35/43] when no more trashData to delete, make continueDeleteImmediately = 0 --- .../org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java | 1 + 1 file changed, 1 insertion(+) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index ca273850a3e4a..a8f536540522d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -318,6 +318,7 @@ private void triggerDelete() { } List toDelete = getToDeleteData(); if (toDelete.size() == 0) { + continueDeleteImmediately.set(0); deleteMutex.unlock(); return; } From b4c54d5d8db47a8a16ba9ba9598963f49d1a0075 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Wed, 8 Jun 2022 21:33:38 +0800 Subject: [PATCH 36/43] 1.make maxDeleteCount support config. 2.add api: allTrashDataDeleteOnce. --- .../mledger/ManagedLedgerConfig.java | 9 +++++ .../bookkeeper/mledger/ManagedTrash.java | 2 ++ .../mledger/impl/ManagedTrashDisableImpl.java | 5 +++ .../mledger/impl/ManagedTrashImpl.java | 22 ++++++++---- .../mledger/impl/ManagedTrashTest.java | 35 ++++++++++++++++++- .../pulsar/broker/ServiceConfiguration.java | 10 +++++- .../pulsar/broker/service/BrokerService.java | 1 + 7 files changed, 76 insertions(+), 8 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 264cb34f54414..1f3ba052ee3d2 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -84,6 +84,7 @@ public class ManagedLedgerConfig { private boolean supportTwoPhaseDeletion = false; private int archiveDataLimitSize = 500; private int deleteIntervalSeconds = 60; + private int maxDeleteCount = 5; public boolean isCreateIfMissing() { return createIfMissing; @@ -709,4 +710,12 @@ public int getDeleteIntervalSeconds() { public void setDeleteIntervalSeconds(int deleteIntervalSeconds) { this.deleteIntervalSeconds = deleteIntervalSeconds; } + + public void setMaxDeleteCount(int maxDeleteCount) { + this.maxDeleteCount = maxDeleteCount; + } + + public int getMaxDeleteCount() { + return maxDeleteCount; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 20c5678f2846d..9a65b06a3d457 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -65,4 +65,6 @@ enum LedgerType { long getToArchiveDataSize(); void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx); + + boolean allTrashDataDeleteOnce(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index 5613a35274168..01ea44e35b2fa 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -78,4 +78,9 @@ public long getToArchiveDataSize() { public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { callback.closeComplete(ctx); } + + @Override + public boolean allTrashDataDeleteOnce() { + return false; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index a8f536540522d..087cd8f11615c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -72,8 +72,6 @@ public class ManagedTrashImpl implements ManagedTrash { private static final String TRASH_KEY_SEPARATOR = "-"; - private static final int RETRY_COUNT = 3; - private static final long EMPTY_LEDGER_ID = -1L; private static final LedgerInfo EMPTY_LEDGER_INFO = LedgerInfo.newBuilder().setLedgerId(EMPTY_LEDGER_ID).build(); @@ -117,6 +115,8 @@ public class ManagedTrashImpl implements ManagedTrash { private final long deleteIntervalMillis; + private final int maxDeleteCount; + private volatile boolean trashIsDirty; private ScheduledFuture checkTrashPersistTask; @@ -134,6 +134,7 @@ public ManagedTrashImpl(ManagedType type, String name, MetadataStore metadataSto this.bookKeeper = bookKeeper; this.archiveDataLimitSize = config.getArchiveDataLimitSize(); this.deleteIntervalMillis = TimeUnit.SECONDS.toMillis(config.getDeleteIntervalSeconds()); + this.maxDeleteCount = Integer.max(1, config.getMaxDeleteCount()); this.managedTrashMXBean = new ManagedTrashMXBeanImpl(this); STATE_UPDATER.set(this, State.None); } @@ -208,9 +209,9 @@ public void appendLedgerTrashData(long ledgerId, LedgerInfo context, LedgerType } TrashKey key = null; if (ManagedTrash.LedgerType.LEDGER.equals(type)) { - key = TrashKey.buildKey(RETRY_COUNT, ledgerId, 0L, type); + key = TrashKey.buildKey(maxDeleteCount, ledgerId, 0L, type); } else if (ManagedTrash.LedgerType.OFFLOAD_LEDGER.equals(type)) { - key = TrashKey.buildKey(RETRY_COUNT, ledgerId, context.getOffloadContext().getUidMsb(), type); + key = TrashKey.buildKey(maxDeleteCount, ledgerId, context.getOffloadContext().getUidMsb(), type); } trashData.put(key, context); managedTrashMXBean.increaseTotalNumberOfDeleteLedgers(); @@ -322,6 +323,7 @@ private void triggerDelete() { deleteMutex.unlock(); return; } + //filter trashData which last time and current time differ by no more than deleteIntervalMillis. toDelete.removeIf(ele -> System.currentTimeMillis() - ele.key.lastDeleteTs < deleteIntervalMillis); List> futures = new ArrayList<>(); @@ -334,6 +336,14 @@ private void triggerDelete() { }); } + @Override + public boolean allTrashDataDeleteOnce() { + if (trashData.isEmpty()) { + return true; + } + return trashData.lastEntry().getKey().retryCount < maxDeleteCount; + } + @Override public CompletableFuture> getAllArchiveIndex() { return metadataStore.getChildren(buildParentPath()).thenComposeAsync(children -> { @@ -566,10 +576,10 @@ private void onDeleteFailed(DelHelper helper) { String info = null; if (helper.key.isLedger()) { info = String.format("[%s] Delete ledger %d reach retry limit %d.", name(), helper.key.ledgerId, - RETRY_COUNT); + maxDeleteCount); } else if (helper.key.isOffloadLedger()) { info = String.format("[%s] Delete offload ledger %d reach retry limit %d.", name(), - helper.key.ledgerId, RETRY_COUNT); + helper.key.ledgerId, maxDeleteCount); } log.warn(info); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java index eb3691a2f78aa..b7b1481b12cfe 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java @@ -19,6 +19,8 @@ package org.apache.bookkeeper.mledger.impl; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; import com.google.common.base.Charsets; import java.lang.reflect.Field; import java.nio.charset.Charset; @@ -197,6 +199,7 @@ public void testManagedTrashArchive() throws Exception { managedLedgerConfig.setSupportTwoPhaseDeletion(true); managedLedgerConfig.setDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); + managedLedgerConfig.setMaxDeleteCount(1); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); @@ -313,6 +316,7 @@ public void testGetAllArchiveIndex() throws Exception { managedLedgerConfig.setSupportTwoPhaseDeletion(true); managedLedgerConfig.setDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); + managedLedgerConfig.setMaxDeleteCount(1); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); @@ -343,6 +347,7 @@ public void testGetArchiveData() throws Exception { managedLedgerConfig.setSupportTwoPhaseDeletion(true); managedLedgerConfig.setDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); + managedLedgerConfig.setMaxDeleteCount(1); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); @@ -377,7 +382,35 @@ public void testGetArchiveData() throws Exception { Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); i++; } - } + @Test + public void testAllTrashDataDeleteOnce() throws Exception { + int entryCount = 30; + int archiveLimit = 10; + + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); + managedLedgerConfig.setMaxDeleteCount(1); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + + //the ledgerId >= 10000, it will delete failed. see line_142. + for (int i = 0; i < entryCount; i++) { + managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); + } + + assertFalse(managedTrash.allTrashDataDeleteOnce()); + + managedTrash.triggerDeleteInBackground(); + + Awaitility.await().untilAsserted(() -> { + assertTrue(managedTrash.allTrashDataDeleteOnce()); + }); + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 25568da9ad8ae..d6064e4a4d5ab 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2525,7 +2525,7 @@ public class ServiceConfiguration implements PulsarConfiguration { @FieldContext( dynamic = true, category = CATEGORY_STORAGE_ML, - doc = "To control managedTrash archive data size, if reach, persist archive data. (Default 500)" + doc = "To control managedTrash archive data size. If reach, persist archive data. (Default 500)" ) private int managedTrashArchiveDataLimitSize = 500; @@ -2536,6 +2536,14 @@ public class ServiceConfiguration implements PulsarConfiguration { ) private int managedTrashDeleteInternalSeconds = 60; + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "The max delete count of delete same ledger or offload ledger in managedTrash. " + + "If a ledger delete count reach it, the ledger will be archived. (Default is 5)" + ) + private int managedLedgerMaxDeleteCount = 5; + @FieldContext( category = CATEGORY_STORAGE_ML, doc = "Evicting cache data by the slowest markDeletedPosition or readPosition. " diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 8b72f38b216ca..8d0e79787ef65 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1594,6 +1594,7 @@ public CompletableFuture getManagedLedgerConfig(TopicName t serviceConfig.isManagedLedgerSupportTwoPhaseDeletion()); managedLedgerConfig.setArchiveDataLimitSize(serviceConfig.getManagedTrashArchiveDataLimitSize()); managedLedgerConfig.setDeleteIntervalSeconds(serviceConfig.getManagedTrashDeleteInternalSeconds()); + managedLedgerConfig.setMaxDeleteCount(serviceConfig.getManagedLedgerMaxDeleteCount()); OffloadPoliciesImpl nsLevelOffloadPolicies = (OffloadPoliciesImpl) policies.map(p -> p.offload_policies).orElse(null); From fc706900bcaa77a7edd52232a8aa50ca39295abf Mon Sep 17 00:00:00 2001 From: horizonzy Date: Fri, 10 Jun 2022 10:53:20 +0800 Subject: [PATCH 37/43] trigger delete after update trashData. --- .../bookkeeper/mledger/impl/ManagedLedgerImpl.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) 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 4132b446d1731..36f6542888712 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 @@ -3088,7 +3088,9 @@ private void offloadLoop(CompletableFuture promise, Queue { + managedTrash.triggerDeleteInBackground(); + }); } catch (ManagedLedgerException e) { log.warn("[{}]-{} Failed to append trash data.", this.name, ledgerId); } @@ -3222,7 +3224,9 @@ private CompletableFuture prepareLedgerInfoForOffloaded(long ledgerId, UUI } else { managedTrash.appendLedgerTrashData(ledgerId, oldInfo, ManagedTrash.LedgerType.OFFLOAD_LEDGER); - managedTrash.asyncUpdateTrashData(); + managedTrash.asyncUpdateTrashData().thenAccept(ignore -> { + managedTrash.triggerDeleteInBackground(); + }); } } LedgerInfo.Builder builder = oldInfo.toBuilder(); @@ -3903,7 +3907,9 @@ protected boolean checkAndCompleteLedgerOpTask(int rc, LedgerHandle lh, Object c } else { try { managedTrash.appendLedgerTrashData(lh.getId(), null, ManagedTrash.LedgerType.LEDGER); - managedTrash.asyncUpdateTrashData(); + managedTrash.asyncUpdateTrashData().thenAccept(ignore -> { + managedTrash.triggerDeleteInBackground(); + }); } catch (ManagedLedgerException e) { log.warn("[{}]-{} Failed to append trash data.", this.name, lh.getId()); } From 3c7303c8ea3cb8d0db5946264a367ddb8140765c Mon Sep 17 00:00:00 2001 From: horizonzy Date: Fri, 10 Jun 2022 22:46:01 +0800 Subject: [PATCH 38/43] make allTrashDataDeleteOnce async. --- .../bookkeeper/mledger/ManagedTrash.java | 3 +- .../mledger/impl/ManagedLedgerImpl.java | 277 +++++++++--------- .../mledger/impl/ManagedTrashDisableImpl.java | 4 +- .../mledger/impl/ManagedTrashImpl.java | 137 +++++++-- .../mledger/impl/ManagedTrashTest.java | 45 ++- 5 files changed, 293 insertions(+), 173 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 9a65b06a3d457..ee6f5871abc4d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -42,6 +42,7 @@ public String getName() { } enum LedgerType { + BOTH, OFFLOAD_LEDGER, LEDGER } @@ -66,5 +67,5 @@ enum LedgerType { void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx); - boolean allTrashDataDeleteOnce(); + CompletableFuture allTrashDataDeleteOnce(); } 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 36f6542888712..639e56aa2ea6a 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 @@ -351,107 +351,101 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); this.managedTrash = config.isSupportTwoPhaseDeletion() ? new ManagedTrashImpl(ManagedTrash.ManagedType.MANAGED_LEDGER, name, metadataStore, config, - scheduledExecutor, executor, bookKeeper) : new ManagedTrashDisableImpl(); + scheduledExecutor, executor, bookKeeper, ledgers) : new ManagedTrashDisableImpl(); } synchronized void initialize(final ManagedLedgerInitializeLedgerCallback callback, final Object ctx) { log.info("Opening managed ledger {}", name); - - managedTrash.initialize().whenComplete((res, e) -> { - if (e != null) { - callback.initializeFailed(new ManagedLedgerException(e)); - } else { // Fetch the list of existing ledgers in the managed ledger - store.getManagedLedgerInfo(name, config.isCreateIfMissing(), config.getProperties(), - new MetaStoreCallback() { - @Override - public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { - ledgersStat = stat; - if (mlInfo.hasTerminatedPosition()) { - state = State.Terminated; - lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); - log.info("[{}] Recovering managed ledger terminated at {}", name, - lastConfirmedEntry); - } - - for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { - ledgers.put(ls.getLedgerId(), ls); - } + store.getManagedLedgerInfo(name, config.isCreateIfMissing(), config.getProperties(), + new MetaStoreCallback() { + @Override + public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { + ledgersStat = stat; + if (mlInfo.hasTerminatedPosition()) { + state = State.Terminated; + lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); + log.info("[{}] Recovering managed ledger terminated at {}", name, + lastConfirmedEntry); + } - if (mlInfo.getPropertiesCount() > 0) { - propertiesMap = Maps.newHashMap(); - for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { - MLDataFormats.KeyValue property = mlInfo.getProperties(i); - propertiesMap.put(property.getKey(), property.getValue()); - } - } - if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.onManagedLedgerPropertiesInitialize(propertiesMap); - } + for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { + ledgers.put(ls.getLedgerId(), ls); + } - // Last ledger stat may be zeroed, we must update it - if (!ledgers.isEmpty()) { - final long id = ledgers.lastKey(); - OpenCallback opencb = (rc, lh, ctx1) -> { - executor.executeOrdered(name, safeRun(() -> { - mbean.endDataLedgerOpenOp(); - if (log.isDebugEnabled()) { - log.debug("[{}] Opened ledger {}: {}", name, id, - BKException.getMessage(rc)); - } - if (rc == BKException.Code.OK) { - LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(id) - .setEntries(lh.getLastAddConfirmed() + 1) - .setSize(lh.getLength()).setTimestamp(clock.millis()).build(); - ledgers.put(id, info); - if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, - lh).thenRun(() -> initializeBookKeeper(callback)) - .exceptionally(ex -> { - callback.initializeFailed( - new ManagedLedgerInterceptException( - ex.getCause())); - return null; - }); - } else { - initializeBookKeeper(callback); - } - } else if (isNoSuchLedgerExistsException(rc)) { - log.warn("[{}] Ledger not found: {}", name, id); - ledgers.remove(id); - initializeBookKeeper(callback); - } else { - log.error("[{}] Failed to open ledger {}: {}", name, id, - BKException.getMessage(rc)); - callback.initializeFailed(createManagedLedgerException(rc)); - return; - } - })); - }; + if (mlInfo.getPropertiesCount() > 0) { + propertiesMap = Maps.newHashMap(); + for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { + MLDataFormats.KeyValue property = mlInfo.getProperties(i); + propertiesMap.put(property.getKey(), property.getValue()); + } + } + if (managedLedgerInterceptor != null) { + managedLedgerInterceptor.onManagedLedgerPropertiesInitialize(propertiesMap); + } + // Last ledger stat may be zeroed, we must update it + if (!ledgers.isEmpty()) { + final long id = ledgers.lastKey(); + OpenCallback opencb = (rc, lh, ctx1) -> { + executor.executeOrdered(name, safeRun(() -> { + mbean.endDataLedgerOpenOp(); if (log.isDebugEnabled()) { - log.debug("[{}] Opening ledger {}", name, id); + log.debug("[{}] Opened ledger {}: {}", name, id, + BKException.getMessage(rc)); } - mbean.startDataLedgerOpenOp(); - bookKeeper.asyncOpenLedger(id, digestType, config.getPassword(), opencb, null); - } else { - initializeBookKeeper(callback); - } - } + if (rc == BKException.Code.OK) { + LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(id) + .setEntries(lh.getLastAddConfirmed() + 1) + .setSize(lh.getLength()).setTimestamp(clock.millis()).build(); + ledgers.put(id, info); + if (managedLedgerInterceptor != null) { + managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, + lh).thenRun(() -> initializeBookKeeper(callback)) + .exceptionally(ex -> { + callback.initializeFailed( + new ManagedLedgerInterceptException( + ex.getCause())); + return null; + }); + } else { + initializeBookKeeper(callback); + } + } else if (isNoSuchLedgerExistsException(rc)) { + log.warn("[{}] Ledger not found: {}", name, id); + ledgers.remove(id); + initializeBookKeeper(callback); + } else { + log.error("[{}] Failed to open ledger {}: {}", name, id, + BKException.getMessage(rc)); + callback.initializeFailed(createManagedLedgerException(rc)); + return; + } + })); + }; - @Override - public void operationFailed(MetaStoreException e) { - if (e instanceof MetadataNotFoundException) { - callback.initializeFailed(new ManagedLedgerNotFoundException(e)); - } else { - callback.initializeFailed(new ManagedLedgerException(e)); - } + if (log.isDebugEnabled()) { + log.debug("[{}] Opening ledger {}", name, id); } - }); + mbean.startDataLedgerOpenOp(); + bookKeeper.asyncOpenLedger(id, digestType, config.getPassword(), opencb, null); + } else { + initializeBookKeeper(callback); + } + } + + @Override + public void operationFailed(MetaStoreException e) { + if (e instanceof MetadataNotFoundException) { + callback.initializeFailed(new ManagedLedgerNotFoundException(e)); + } else { + callback.initializeFailed(new ManagedLedgerException(e)); + } + } + }); + + scheduleTimeoutTask(); - scheduleTimeoutTask(); - } - }); } private synchronized void initializeBookKeeper(final ManagedLedgerInitializeLedgerCallback callback) { @@ -475,12 +469,22 @@ private synchronized void initializeBookKeeper(final ManagedLedgerInitializeLedg }, null); } } - if (state == State.Terminated) { // When recovering a terminated managed ledger, we don't need to create // a new ledger for writing, since no more writes are allowed. // We just move on to the next stage - initializeCursors(callback); + List> futures = new ArrayList<>(2); + futures.add(initializeCursors()); + futures.add(managedTrash.initialize()); + + FutureUtil.waitForAll(futures).whenComplete((res, e) -> { + if (e != null) { + callback.initializeFailed((ManagedLedgerException) e); + return; + } + callback.initializeComplete(); + }); + return; } @@ -488,7 +492,17 @@ private synchronized void initializeBookKeeper(final ManagedLedgerInitializeLedg @Override public void operationComplete(Void v, Stat stat) { ledgersStat = stat; - initializeCursors(callback); + List> futures = new ArrayList<>(2); + futures.add(initializeCursors()); + futures.add(managedTrash.initialize()); + + FutureUtil.waitForAll(futures).whenComplete((res, e) -> { + if (e != null) { + callback.initializeFailed((ManagedLedgerException) e); + return; + } + callback.initializeComplete(); + }); } @Override @@ -540,7 +554,8 @@ public void operationFailed(MetaStoreException e) { }, ledgerMetadata); } - private void initializeCursors(final ManagedLedgerInitializeLedgerCallback callback) { + private CompletableFuture initializeCursors() { + CompletableFuture future = new CompletableFuture<>(); if (log.isDebugEnabled()) { log.debug("[{}] initializing cursors", name); } @@ -554,7 +569,7 @@ public void operationComplete(List consumers, Stat s) { } if (consumers.isEmpty()) { - callback.initializeComplete(); + future.complete(null); return; } @@ -576,7 +591,7 @@ public void operationComplete() { if (cursorCount.decrementAndGet() == 0) { // The initialization is now completed, register the jmx mbean - callback.initializeComplete(); + future.complete(null); } } @@ -584,7 +599,7 @@ public void operationComplete() { public void operationFailed(ManagedLedgerException exception) { log.warn("[{}] Recovery for cursor {} failed", name, cursorName, exception); cursorCount.set(-1); - callback.initializeFailed(exception); + future.completeExceptionally(exception); } }); } @@ -621,16 +636,17 @@ public void operationFailed(ManagedLedgerException exception) { }); } // Complete ledger recovery. - callback.initializeComplete(); + future.complete(null); } } @Override public void operationFailed(MetaStoreException e) { log.warn("[{}] Failed to get the cursors list", name, e); - callback.initializeFailed(new ManagedLedgerException(e)); + future.completeExceptionally(new ManagedLedgerException(e)); } }); + return future; } @Override @@ -2784,6 +2800,7 @@ public void deleteLedgerFailed(ManagedLedgerException e, Object ctx) { public void asyncDelete(final DeleteLedgerCallback callback, final Object ctx) { // Delete the managed ledger without closing, since we are not interested in gracefully closing cursors and // ledgers + STATE_UPDATER.set(this, State.Fenced); cancelScheduledTasks(); @@ -2874,61 +2891,59 @@ private void asyncDeleteLedger(long ledgerId, long retry) { @SuppressWarnings("checkstyle:fallthrough") private void deleteAllLedgers(DeleteLedgerCallback callback, Object ctx) { List ledgers = Lists.newArrayList(ManagedLedgerImpl.this.ledgers.values()); - AtomicInteger ledgersToDelete = new AtomicInteger(ledgers.size()); if (ledgers.isEmpty()) { // No ledgers to delete, proceed with deleting metadata - deleteMetadata(callback, ctx); + deleteMetadata().whenComplete((res, e) -> { + if (e != null) { + callback.deleteLedgerFailed((ManagedLedgerException) e, ctx); + return; + } + callback.deleteLedgerComplete(ctx); + }); return; } + // Mark deletable ledgers + Set deletableLedgers = ledgers.stream().filter(ls -> !ls.getOffloadContext().getBookkeeperDeleted()) + .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); - for (LedgerInfo ls : ledgers) { - if (log.isDebugEnabled()) { - log.debug("[{}] Deleting ledger {}", name, ls); - } - bookKeeper.asyncDeleteLedger(ls.getLedgerId(), (rc, ctx1) -> { - switch (rc) { - case Code.NoSuchLedgerExistsException: - case Code.NoSuchLedgerExistsOnMetadataServerException: - log.warn("[{}] Ledger {} not found when deleting it", name, ls.getLedgerId()); - // Continue anyway - - case BKException.Code.OK: - if (ledgersToDelete.decrementAndGet() == 0) { - // All ledgers deleted, now remove ML metadata - deleteMetadata(callback, ctx); - } - break; + // Mark deletable offloaded ledgers + Set deletableOffloadedLedgers = ledgers.stream() + .filter(ls -> ls.getOffloadContext().hasUidMsb()) + .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); - default: - // Handle error - log.warn("[{}] Failed to delete ledger {} -- {}", name, ls.getLedgerId(), - BKException.getMessage(rc)); - int toDelete = ledgersToDelete.get(); - if (toDelete != -1 && ledgersToDelete.compareAndSet(toDelete, -1)) { - // Trigger callback only once - callback.deleteLedgerFailed(createManagedLedgerException(rc), ctx); - } - } - }, null); - } + asyncUpdateTrashData(deletableLedgers, deletableOffloadedLedgers).thenCompose(ignore -> { + managedTrash.triggerDeleteInBackground(); + return deleteMetadata(); + }).thenAccept(ignore -> managedTrash.allTrashDataDeleteOnce().whenComplete((res, e) -> { + if (e != null) { + log.error("[{}] ManagedTrash all trash data delete once exception.", name, e); + } + callback.deleteLedgerComplete(ctx); + })).exceptionally(e -> { + callback.deleteLedgerFailed(createManagedLedgerException(e), ctx); + return null; + }); } - private void deleteMetadata(DeleteLedgerCallback callback, Object ctx) { + private CompletableFuture deleteMetadata() { + CompletableFuture future = new CompletableFuture<>(); store.removeManagedLedger(name, new MetaStoreCallback() { @Override public void operationComplete(Void result, Stat stat) { log.info("[{}] Successfully deleted managed ledger", name); factory.close(ManagedLedgerImpl.this); - callback.deleteLedgerComplete(ctx); + ledgers.clear(); + future.complete(null); } @Override public void operationFailed(MetaStoreException e) { log.warn("[{}] Failed to delete managed ledger", name, e); factory.close(ManagedLedgerImpl.this); - callback.deleteLedgerFailed(e, ctx); + future.completeExceptionally(e); } }); + return future; } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index 01ea44e35b2fa..f4ec2c7041c48 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -80,7 +80,7 @@ public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { } @Override - public boolean allTrashDataDeleteOnce() { - return false; + public CompletableFuture allTrashDataDeleteOnce() { + return COMPLETABLE_FUTURE; } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 087cd8f11615c..601af6e49ec51 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -123,12 +123,16 @@ public class ManagedTrashImpl implements ManagedTrash { private final ManagedTrashMXBean managedTrashMXBean; + private final NavigableMap managedLedgers; + public ManagedTrashImpl(ManagedType type, String name, MetadataStore metadataStore, ManagedLedgerConfig config, - OrderedScheduler scheduledExecutor, OrderedExecutor executor, BookKeeper bookKeeper) { + OrderedScheduler scheduledExecutor, OrderedExecutor executor, BookKeeper bookKeeper, + NavigableMap managedLedgers) { this.type = type.getName(); this.name = name; this.config = config; this.metadataStore = metadataStore; + this.managedLedgers = managedLedgers; this.scheduledExecutor = scheduledExecutor; this.executor = executor; this.bookKeeper = bookKeeper; @@ -150,7 +154,7 @@ public CompletableFuture initialize() { metadataStore.get(buildDeletePath()).whenCompleteAsync((res, e) -> { if (e != null) { log.error("[{}] Get delete data failed.", name(), e); - future.completeExceptionally(e); + future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(e)); return; } if (res.isEmpty()) { @@ -175,7 +179,7 @@ public CompletableFuture initialize() { } catch (InvalidProtocolBufferException exc) { future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(exc)); } - }, scheduledExecutor.chooseThread(name)); + }, executor.chooseThread(name)); return future; } @@ -200,7 +204,7 @@ private int calculateArchiveCount() { public void appendLedgerTrashData(long ledgerId, LedgerInfo context, LedgerType type) throws ManagedLedgerException { State state = STATE_UPDATER.get(this); - if (state != State.Initialized) { + if (State.FENCED == state || State.Closed == state) { throw ManagedLedgerException.getManagedLedgerException(new IllegalStateException( String.format("[%s] is not initialized, current state: %s", name(), state))); } @@ -233,9 +237,9 @@ public void asyncUpdateTrashDataInBackground(CompletableFuture future) { private void doAsyncUpdateTrashData(CompletableFuture future) { State state = STATE_UPDATER.get(this); - if (state != State.Initialized) { - future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(new IllegalStateException( - String.format("[%s] is not initialized, current state: %s", name(), state)))); + if (State.Closed == state) { + future.completeExceptionally(ManagedLedgerException.getManagedLedgerException( + new IllegalStateException(String.format("[%s] is closed.", name())))); return; } if (!trashMutex.tryLock()) { @@ -304,30 +308,41 @@ private NavigableMap transferFrom(Map @Override public void triggerDeleteInBackground() { - executor.executeOrdered(name, safeRun(this::triggerDelete)); + State state = STATE_UPDATER.get(this); + LedgerType type; + //if state fenced, means that it only delete ledgers + if (State.FENCED == state) { + type = LedgerType.LEDGER; + } else { + type = LedgerType.BOTH; + } + executor.executeOrdered(name, safeRun(() -> triggerDelete(type))); } - private void triggerDelete() { + + private void triggerDelete(LedgerType type) { State state = STATE_UPDATER.get(this); - if (state != State.Initialized) { - log.warn("[{}] is not initialized, current state: {}", name(), state); + if (State.Closed == state) { + log.warn("[{}] is closed", name()); return; } if (!deleteMutex.tryLock()) { continueDeleteImmediately.incrementAndGet(); return; } - List toDelete = getToDeleteData(); - if (toDelete.size() == 0) { - continueDeleteImmediately.set(0); + Tuple tuple = getToDeleteData(type); + if (tuple.toDelete.size() == 0) { deleteMutex.unlock(); + //if filtered, means that there still + if (tuple.filtered) { + continueDeleteIfNecessary(); + } else { + continueDeleteImmediately.set(0); + } return; } - //filter trashData which last time and current time differ by no more than deleteIntervalMillis. - toDelete.removeIf(ele -> System.currentTimeMillis() - ele.key.lastDeleteTs < deleteIntervalMillis); - List> futures = new ArrayList<>(); - for (DelHelper delHelper : toDelete) { + for (DelHelper delHelper : tuple.toDelete) { futures.add(asyncDeleteTrash(delHelper)); } FutureUtil.waitForAll(futures).whenCompleteAsync((res, e) -> { @@ -337,11 +352,51 @@ private void triggerDelete() { } @Override - public boolean allTrashDataDeleteOnce() { + public CompletableFuture allTrashDataDeleteOnce() { + //ensure can't add more trashData. + STATE_UPDATER.set(this, State.FENCED); + CompletableFuture future = new CompletableFuture<>(); + allTrashDataDeleteOnce(future); + return future.thenCompose(ignore -> { + CompletableFuture finalFuture = new CompletableFuture<>(); + asyncClose(new AsyncCallbacks.CloseCallback() { + @Override + public void closeComplete(Object ctx) { + finalFuture.complete(null); + } + + @Override + public void closeFailed(ManagedLedgerException exception, Object ctx) { + finalFuture.completeExceptionally(exception); + } + }, null); + return finalFuture; + }); + } + + private void allTrashDataDeleteOnce(CompletableFuture future) { if (trashData.isEmpty()) { - return true; + future.complete(null); + return; } - return trashData.lastEntry().getKey().retryCount < maxDeleteCount; + Optional> lastLedgerTrashData = + trashData.descendingMap().entrySet().stream().filter(ele -> + LedgerType.OFFLOAD_LEDGER != ele.getKey().getType()).findFirst(); + if (lastLedgerTrashData.isEmpty()) { + future.complete(null); + return; + } + if (lastLedgerTrashData.get().getKey().getRetryCount() < maxDeleteCount) { + future.complete(null); + return; + } + triggerDeleteInBackground(); + scheduledExecutor.schedule(() -> allTrashDataDeleteOnceInBackground(future), 1000, + TimeUnit.MILLISECONDS); + } + + private void allTrashDataDeleteOnceInBackground(CompletableFuture future) { + executor.executeOrdered(name, safeRun(() -> allTrashDataDeleteOnce(future))); } @Override @@ -394,7 +449,7 @@ public long getToArchiveDataSize() { @Override public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { - if (STATE_UPDATER.get(this) == State.Closed) { + if (State.Closed == STATE_UPDATER.get(this)) { callback.closeComplete(ctx); return; } @@ -441,9 +496,9 @@ private String buildArchivePath(long index) { } //take 1/10 trash to delete, if the size over 10, use 10 to delete. - private List getToDeleteData() { + private Tuple getToDeleteData(LedgerType type) { if (trashData.size() == 0) { - return Collections.emptyList(); + return new Tuple(Collections.emptyList(), false); } int batchSize = trashData.size() / 10; if (batchSize > 10) { @@ -452,18 +507,41 @@ private List getToDeleteData() { if (batchSize == 0) { batchSize = 1; } + boolean filtered = false; List toDelete = new ArrayList<>(batchSize); + for (Map.Entry entry : trashData.descendingMap().entrySet()) { //if last retryCount is zero, the before data retryCount is zero too. if (entry.getKey().retryCount == 0) { break; } + if (LedgerType.BOTH != type && type != entry.getKey().getType()) { + continue; + } + //1.filter trashData which last time and current time differ by no more than deleteIntervalMillis. + //2.filter trashData which still exist in managedLedgers + if (System.currentTimeMillis() - entry.getKey().lastDeleteTs < deleteIntervalMillis + || managedLedgers.containsKey(entry.getKey().getLedgerId())) { + filtered = true; + continue; + } + toDelete.add(DelHelper.buildHelper(entry.getKey(), entry.getValue())); if (toDelete.size() == batchSize) { break; } } - return toDelete; + return new Tuple(toDelete, filtered); + } + + private static class Tuple { + private List toDelete; + private boolean filtered; + + public Tuple(List toDelete, boolean filtered) { + this.toDelete = toDelete; + this.filtered = filtered; + } } private void asyncUpdateArchiveData(CompletableFuture future) { @@ -474,9 +552,9 @@ private void asyncUpdateArchiveData(CompletableFuture future) { log.info("[{}] Start async update archive data", name()); State state = STATE_UPDATER.get(this); - if (state != State.Initialized) { - future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(new IllegalStateException( - String.format("[%s] is not initialized, current state: %s", name(), state)))); + if (State.Closed == state) { + future.completeExceptionally(ManagedLedgerException.getManagedLedgerException( + new IllegalStateException(String.format("[%s] is closed.", name())))); return; } asyncUpdateTrashData().thenAccept(ignore -> { @@ -589,7 +667,7 @@ private void onDeleteFailed(DelHelper helper) { private void continueDeleteIfNecessary() { Map.Entry lastEntry = trashData.lastEntry(); - if (lastEntry == null) { + if (trashData.isEmpty()) { return; } if (lastEntry.getKey().retryCount > 0) { @@ -772,6 +850,7 @@ public int compareTo(TrashKey other) { public enum State { None, Initialized, + FENCED, Closed, } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java index b7b1481b12cfe..240397a95c9b2 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java @@ -19,7 +19,6 @@ package org.apache.bookkeeper.mledger.impl; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import com.google.common.base.Charsets; import java.lang.reflect.Field; @@ -199,7 +198,7 @@ public void testManagedTrashArchive() throws Exception { managedLedgerConfig.setSupportTwoPhaseDeletion(true); managedLedgerConfig.setDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); - managedLedgerConfig.setMaxDeleteCount(1); + managedLedgerConfig.setMaxDeleteCount(3); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); @@ -316,7 +315,7 @@ public void testGetAllArchiveIndex() throws Exception { managedLedgerConfig.setSupportTwoPhaseDeletion(true); managedLedgerConfig.setDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); - managedLedgerConfig.setMaxDeleteCount(1); + managedLedgerConfig.setMaxDeleteCount(3); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); @@ -347,7 +346,7 @@ public void testGetArchiveData() throws Exception { managedLedgerConfig.setSupportTwoPhaseDeletion(true); managedLedgerConfig.setDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); - managedLedgerConfig.setMaxDeleteCount(1); + managedLedgerConfig.setMaxDeleteCount(3); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); @@ -388,12 +387,13 @@ public void testGetArchiveData() throws Exception { public void testAllTrashDataDeleteOnce() throws Exception { int entryCount = 30; int archiveLimit = 10; + int maxDeleteCount = 3; ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setSupportTwoPhaseDeletion(true); managedLedgerConfig.setDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); - managedLedgerConfig.setMaxDeleteCount(1); + managedLedgerConfig.setMaxDeleteCount(maxDeleteCount); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); @@ -405,12 +405,37 @@ public void testAllTrashDataDeleteOnce() throws Exception { managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); } - assertFalse(managedTrash.allTrashDataDeleteOnce()); + try { + managedTrash.allTrashDataDeleteOnce().get(); + } catch (Exception e) { + Assert.fail(); + } + assertEquals(managedTrash.state, ManagedTrashImpl.State.Closed); - managedTrash.triggerDeleteInBackground(); + Field field1 = ManagedTrashImpl.class.getDeclaredField("trashData"); + field1.setAccessible(true); + ConcurrentSkipListMap trashData = + (ConcurrentSkipListMap) field1.get(managedTrash); - Awaitility.await().untilAsserted(() -> { - assertTrue(managedTrash.allTrashDataDeleteOnce()); - }); + assertEquals(trashData.size(), 30); + + Optional> lastLedgerTrashData = + trashData.descendingMap().entrySet().stream() + .filter(ele -> ele.getKey().getType() != ManagedTrash.LedgerType.OFFLOAD_LEDGER) + .findFirst(); + assertTrue(lastLedgerTrashData.isPresent()); + assertTrue(lastLedgerTrashData.get().getKey().getRetryCount() < maxDeleteCount); + + byte[] content = persistedData.get("/managed-trash/managed-ledger/my_test_ledger/delete"); + + NavigableMap persistedTrashData = managedTrash.deSerialize(content); + + assertEquals(persistedTrashData.size(), 30); + + for (int i = 0; i < persistedTrashData.size(); i++) { + Map.Entry entry1 = persistedTrashData.pollFirstEntry(); + Map.Entry entry2 = trashData.pollFirstEntry(); + assertEquals(entry1.getKey().compareTo(entry2.getKey()), 0); + } } } From 056372a4f0f5259d6157b0f5d7c1cc7bc6b6537d Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 13 Jun 2022 10:50:27 +0800 Subject: [PATCH 39/43] add test case for getToDeleteData --- .../bookkeeper/mledger/ManagedTrash.java | 2 +- .../mledger/impl/ManagedLedgerImpl.java | 2 +- .../mledger/impl/ManagedTrashDisableImpl.java | 2 +- .../mledger/impl/ManagedTrashImpl.java | 43 ++- .../mledger/impl/ManagedTrashTest.java | 253 ++++++++++++++++-- 5 files changed, 276 insertions(+), 26 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index ee6f5871abc4d..55c61d4954f46 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -67,5 +67,5 @@ enum LedgerType { void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx); - CompletableFuture allTrashDataDeleteOnce(); + CompletableFuture asyncCloseAfterAllTrashDataDeleteOnce(); } 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 639e56aa2ea6a..66c6939e420c5 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 @@ -2914,7 +2914,7 @@ private void deleteAllLedgers(DeleteLedgerCallback callback, Object ctx) { asyncUpdateTrashData(deletableLedgers, deletableOffloadedLedgers).thenCompose(ignore -> { managedTrash.triggerDeleteInBackground(); return deleteMetadata(); - }).thenAccept(ignore -> managedTrash.allTrashDataDeleteOnce().whenComplete((res, e) -> { + }).thenAccept(ignore -> managedTrash.asyncCloseAfterAllTrashDataDeleteOnce().whenComplete((res, e) -> { if (e != null) { log.error("[{}] ManagedTrash all trash data delete once exception.", name, e); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index f4ec2c7041c48..25e73d70b396d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -80,7 +80,7 @@ public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { } @Override - public CompletableFuture allTrashDataDeleteOnce() { + public CompletableFuture asyncCloseAfterAllTrashDataDeleteOnce() { return COMPLETABLE_FUTURE; } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 601af6e49ec51..3a70c525f17e5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -262,7 +262,7 @@ private void doAsyncUpdateTrashData(CompletableFuture future) { } - public byte[] serialize(Map toPersist) { + public static byte[] serialize(Map toPersist) { Map transfer = transferTo(toPersist); TrashDataComponent.Builder builder = TrashDataComponent.newBuilder(); for (Map.Entry entry : transfer.entrySet()) { @@ -275,7 +275,7 @@ public byte[] serialize(Map toPersist) { return builder.build().toByteArray(); } - private Map transferTo(Map to) { + private static Map transferTo(Map to) { Map result = new ConcurrentSkipListMap<>(); for (Map.Entry entry : to.entrySet()) { result.put(entry.getKey().toStringKey(), entry.getValue()); @@ -283,7 +283,7 @@ private Map transferTo(Map to) { return result; } - public NavigableMap deSerialize(byte[] content) throws InvalidProtocolBufferException { + public static NavigableMap deSerialize(byte[] content) throws InvalidProtocolBufferException { TrashDataComponent component = TrashDataComponent.parseFrom(content); List componentList = component.getComponentList(); Map result = new ConcurrentSkipListMap<>(); @@ -298,7 +298,7 @@ public NavigableMap deSerialize(byte[] content) throws Inv } - private NavigableMap transferFrom(Map from) { + private static NavigableMap transferFrom(Map from) { NavigableMap result = new ConcurrentSkipListMap<>(); for (Map.Entry entry : from.entrySet()) { result.put(TrashKey.buildKey(entry.getKey()), entry.getValue()); @@ -352,7 +352,7 @@ private void triggerDelete(LedgerType type) { } @Override - public CompletableFuture allTrashDataDeleteOnce() { + public CompletableFuture asyncCloseAfterAllTrashDataDeleteOnce() { //ensure can't add more trashData. STATE_UPDATER.set(this, State.FENCED); CompletableFuture future = new CompletableFuture<>(); @@ -496,7 +496,7 @@ private String buildArchivePath(long index) { } //take 1/10 trash to delete, if the size over 10, use 10 to delete. - private Tuple getToDeleteData(LedgerType type) { + protected Tuple getToDeleteData(LedgerType type) { if (trashData.size() == 0) { return new Tuple(Collections.emptyList(), false); } @@ -534,7 +534,7 @@ private Tuple getToDeleteData(LedgerType type) { return new Tuple(toDelete, filtered); } - private static class Tuple { + static class Tuple { private List toDelete; private boolean filtered; @@ -542,6 +542,14 @@ public Tuple(List toDelete, boolean filtered) { this.toDelete = toDelete; this.filtered = filtered; } + + public List getToDelete() { + return toDelete; + } + + public boolean isFiltered() { + return filtered; + } } private void asyncUpdateArchiveData(CompletableFuture future) { @@ -746,7 +754,7 @@ private static ManagedLedgerException.MetaStoreException getMetaStoreException(T } } - private static class DelHelper { + protected static class DelHelper { private final TrashKey key; private final LedgerInfo context; @@ -755,6 +763,14 @@ public DelHelper(TrashKey key, LedgerInfo context) { this.context = context; } + public TrashKey getKey() { + return key; + } + + public LedgerInfo getContext() { + return context; + } + public static DelHelper buildHelper(TrashKey key, LedgerInfo context) { return new DelHelper(key, context); } @@ -845,6 +861,17 @@ public int compareTo(TrashKey other) { } return this.type.compareTo(other.type); } + + @Override + public String toString() { + return "TrashKey{" + + "retryCount=" + retryCount + + ", ledgerId=" + ledgerId + + ", msb=" + msb + + ", type=" + type + + ", lastDeleteTs=" + lastDeleteTs + + '}'; + } } public enum State { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java index 240397a95c9b2..6e8eed3ffa499 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java @@ -79,6 +79,7 @@ public CompletableFuture put(String path, byte[] value, Optional exp }); return future; } + }; bkc = new PulsarMockBookKeeper(executor) { @Override @@ -113,6 +114,14 @@ protected void cleanUpTestCase() throws Exception { persistedData.clear(); } + private LedgerInfo buildOffloadLedgerInfo(long ledgerId, UUID uuid) { + LedgerInfo.Builder builder = LedgerInfo.newBuilder().setLedgerId(ledgerId); + builder.getOffloadContextBuilder() + .setUidMsb(uuid.getMostSignificantBits()) + .setUidLsb(uuid.getLeastSignificantBits()); + return builder.build(); + } + @Test public void testTrashKeyOrder() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); @@ -122,6 +131,7 @@ public void testTrashKeyOrder() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); managedTrash.appendLedgerTrashData(10, null, ManagedTrash.LedgerType.LEDGER); @@ -131,17 +141,14 @@ public void testTrashKeyOrder() throws Exception { managedTrash.appendLedgerTrashData(7, null, ManagedTrash.LedgerType.LEDGER); - LedgerInfo.Builder builder = LedgerInfo.newBuilder().setLedgerId(7); UUID uuid = UUID.randomUUID(); - builder.getOffloadContextBuilder() - .setUidMsb(uuid.getMostSignificantBits()) - .setUidLsb(uuid.getLeastSignificantBits()); - LedgerInfo ledgerInfo = builder.build(); + LedgerInfo ledgerInfo = buildOffloadLedgerInfo(7, uuid); managedTrash.appendLedgerTrashData(7, ledgerInfo, ManagedTrash.LedgerType.OFFLOAD_LEDGER); Field field1 = ManagedTrashImpl.class.getDeclaredField("trashData"); field1.setAccessible(true); ConcurrentSkipListMap map = (ConcurrentSkipListMap) field1.get(managedTrash); + System.out.println(map); Map.Entry entry = map.pollFirstEntry(); Assert.assertEquals(entry.getKey().getLedgerId(), 3); @@ -153,15 +160,27 @@ public void testTrashKeyOrder() throws Exception { Assert.assertEquals(entry.getKey().getMsb(), 0); Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); - entry = map.pollFirstEntry(); - Assert.assertEquals(entry.getKey().getLedgerId(), 7); - Assert.assertEquals(entry.getKey().getMsb(), uuid.getMostSignificantBits()); - Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.OFFLOAD_LEDGER); + if (uuid.getMostSignificantBits() > 0) { + entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 7); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); - entry = map.pollFirstEntry(); - Assert.assertEquals(entry.getKey().getLedgerId(), 7); - Assert.assertEquals(entry.getKey().getMsb(), 0); - Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 7); + Assert.assertEquals(entry.getKey().getMsb(), uuid.getMostSignificantBits()); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.OFFLOAD_LEDGER); + } else { + entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 7); + Assert.assertEquals(entry.getKey().getMsb(), uuid.getMostSignificantBits()); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.OFFLOAD_LEDGER); + + entry = map.pollFirstEntry(); + Assert.assertEquals(entry.getKey().getLedgerId(), 7); + Assert.assertEquals(entry.getKey().getMsb(), 0); + Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); + } entry = map.pollFirstEntry(); Assert.assertEquals(entry.getKey().getLedgerId(), 10); @@ -174,6 +193,205 @@ public void testTrashKeyOrder() throws Exception { Assert.assertEquals(entry.getKey().getType(), ManagedTrash.LedgerType.LEDGER); } + + @Test + public void testGetToDeleteDataFilterByManagedLedgers() throws Exception { + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setMaxEntriesPerLedger(10); + managedLedgerConfig.setRetentionTime(3, TimeUnit.SECONDS); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + Field field1 = ManagedTrashImpl.class.getDeclaredField("managedLedgers"); + field1.setAccessible(true); + + NavigableMap managedLedgers = (NavigableMap) field1.get(managedTrash); + + LedgerInfo emptyLedgerInfo = LedgerInfo.newBuilder().setLedgerId(-1).build(); + for (int i = 0; i < 30; i++) { + long ledgerId = 10000 + i; + managedLedgers.put(ledgerId, emptyLedgerInfo); + if (i % 2 == 0) { + managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.LedgerType.LEDGER); + } else { + //build offload ledger + UUID uuid = UUID.randomUUID(); + LedgerInfo ledgerInfo = buildOffloadLedgerInfo(ledgerId, uuid); + managedTrash.appendLedgerTrashData(ledgerId, ledgerInfo, ManagedTrash.LedgerType.OFFLOAD_LEDGER); + } + } + ManagedTrashImpl.Tuple tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.BOTH); + assertEquals(tuple.getToDelete().size(), 0); + assertEquals(tuple.isFiltered(), true); + + //when managedLedgers remove 29, it can be to delete. + managedLedgers.remove(10029L); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.BOTH); + assertEquals(tuple.getToDelete().size(), 1); + assertEquals(tuple.isFiltered(), true); + assertEquals(tuple.getToDelete().get(0).getKey().getLedgerId(), 10029L); + assertEquals(tuple.getToDelete().get(0).getKey().getType(), ManagedTrash.LedgerType.OFFLOAD_LEDGER); + + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.LEDGER); + assertEquals(tuple.getToDelete().size(), 0); + assertEquals(tuple.isFiltered(), true); + + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.OFFLOAD_LEDGER); + assertEquals(tuple.getToDelete().size(), 1); + assertEquals(tuple.isFiltered(), true); + assertEquals(tuple.getToDelete().get(0).getKey().getLedgerId(), 10029L); + assertEquals(tuple.getToDelete().get(0).getKey().getType(), ManagedTrash.LedgerType.OFFLOAD_LEDGER); + + managedLedgers.remove(10028L); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.BOTH); + assertEquals(tuple.getToDelete().size(), 2); + assertEquals(tuple.isFiltered(), true); + assertEquals(tuple.getToDelete().get(0).getKey().getLedgerId(), 10029L); + assertEquals(tuple.getToDelete().get(0).getKey().getType(), ManagedTrash.LedgerType.OFFLOAD_LEDGER); + assertEquals(tuple.getToDelete().get(1).getKey().getLedgerId(), 10028L); + assertEquals(tuple.getToDelete().get(1).getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.LEDGER); + assertEquals(tuple.getToDelete().size(), 1); + assertEquals(tuple.isFiltered(), true); + assertEquals(tuple.getToDelete().get(0).getKey().getLedgerId(), 10028L); + assertEquals(tuple.getToDelete().get(0).getKey().getType(), ManagedTrash.LedgerType.LEDGER); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.OFFLOAD_LEDGER); + assertEquals(tuple.getToDelete().size(), 1); + assertEquals(tuple.isFiltered(), true); + assertEquals(tuple.getToDelete().get(0).getKey().getLedgerId(), 10029L); + assertEquals(tuple.getToDelete().get(0).getKey().getType(), ManagedTrash.LedgerType.OFFLOAD_LEDGER); + } + + @Test + public void testGetToDeleteDataFilterByLastTimeStamp() throws Exception { + int maxDeleteCount = 3; + int deleteIntervalSeconds = 5; + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setMaxEntriesPerLedger(10); + managedLedgerConfig.setRetentionTime(3, TimeUnit.SECONDS); + managedLedgerConfig.setMaxDeleteCount(maxDeleteCount); + managedLedgerConfig.setDeleteIntervalSeconds(deleteIntervalSeconds); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + Field field1 = ManagedTrashImpl.class.getDeclaredField("managedLedgers"); + field1.setAccessible(true); + + for (int i = 0; i < 30; i++) { + long ledgerId = 10000 + i; + if (i % 2 == 0) { + managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.LedgerType.LEDGER); + } else { + //build offload ledger + UUID uuid = UUID.randomUUID(); + LedgerInfo ledgerInfo = buildOffloadLedgerInfo(ledgerId, uuid); + managedTrash.appendLedgerTrashData(ledgerId, ledgerInfo, ManagedTrash.LedgerType.OFFLOAD_LEDGER); + } + } + ManagedTrashImpl.Tuple tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.BOTH); + assertEquals(tuple.getToDelete().size(), 3); + assertEquals(tuple.isFiltered(), false); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.LEDGER); + assertEquals(tuple.getToDelete().size(), 3); + assertEquals(tuple.isFiltered(), false); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.OFFLOAD_LEDGER); + assertEquals(tuple.getToDelete().size(), 3); + assertEquals(tuple.isFiltered(), false); + + for (int i = 0; i < 10; i++) { + managedTrash.triggerDeleteInBackground(); + } + + + Field field2 = ManagedTrashImpl.class.getDeclaredField("trashData"); + field2.setAccessible(true); + ConcurrentSkipListMap trashData = + (ConcurrentSkipListMap) field2.get(managedTrash); + + Awaitility.await().untilAsserted(() -> { + assertEquals(trashData.lastEntry().getKey().getRetryCount(), maxDeleteCount - 1); + }); + + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.BOTH); + assertEquals(tuple.getToDelete().size(), 0); + assertEquals(tuple.isFiltered(), true); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.LEDGER); + assertEquals(tuple.getToDelete().size(), 0); + assertEquals(tuple.isFiltered(), true); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.OFFLOAD_LEDGER); + assertEquals(tuple.getToDelete().size(), 0); + assertEquals(tuple.isFiltered(), true); + + Thread.sleep(deleteIntervalSeconds * 1000); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.BOTH); + assertEquals(tuple.getToDelete().size(), 3); + assertEquals(tuple.isFiltered(), false); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.LEDGER); + assertEquals(tuple.getToDelete().size(), 3); + assertEquals(tuple.isFiltered(), false); + + tuple = managedTrash.getToDeleteData(ManagedTrash.LedgerType.OFFLOAD_LEDGER); + assertEquals(tuple.getToDelete().size(), 3); + assertEquals(tuple.isFiltered(), false); + } + + @Test + public void testRecover() throws Exception { + int entryCount = 30; + ConcurrentSkipListMap map = new ConcurrentSkipListMap<>(); + LedgerInfo emptyLedgerInfo = LedgerInfo.newBuilder().setLedgerId(-1).build(); + for (int i = 0; i < entryCount; i++) { + ManagedTrashImpl.TrashKey trashKey = new ManagedTrashImpl.TrashKey(3, i, 0, ManagedTrash.LedgerType.LEDGER); + map.put(trashKey, emptyLedgerInfo); + } + metadataStore.put("/managed-trash/managed-ledger/my_test_ledger/delete", ManagedTrashImpl.serialize(map), + Optional.of(-1L)); + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setMaxEntriesPerLedger(10); + managedLedgerConfig.setRetentionTime(1, TimeUnit.SECONDS); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + Field field1 = ManagedTrashImpl.class.getDeclaredField("trashData"); + field1.setAccessible(true); + ConcurrentSkipListMap trashData = + (ConcurrentSkipListMap) field1.get(managedTrash); + + assertEquals(trashData.size(), entryCount); + + for (int i = 0; i < entryCount; i++) { + Map.Entry entry1 = map.pollFirstEntry(); + Map.Entry entry2 = trashData.pollFirstEntry(); + assertEquals(entry1.getKey().getLedgerId(), entry2.getKey().getLedgerId()); + } + } + + @Test public void testManagedTrashDelete() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); @@ -202,6 +420,7 @@ public void testManagedTrashArchive() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); @@ -258,6 +477,7 @@ public void testManagedTrashClose() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); managedTrash.appendLedgerTrashData(10, null, ManagedTrash.LedgerType.LEDGER); @@ -319,6 +539,7 @@ public void testGetAllArchiveIndex() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); @@ -350,6 +571,7 @@ public void testGetArchiveData() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); @@ -384,7 +606,7 @@ public void testGetArchiveData() throws Exception { } @Test - public void testAllTrashDataDeleteOnce() throws Exception { + public void testAsyncCloseAfterAllTrashDataDeleteOnce() throws Exception { int entryCount = 30; int archiveLimit = 10; int maxDeleteCount = 3; @@ -397,6 +619,7 @@ public void testAllTrashDataDeleteOnce() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); @@ -406,7 +629,7 @@ public void testAllTrashDataDeleteOnce() throws Exception { } try { - managedTrash.allTrashDataDeleteOnce().get(); + managedTrash.asyncCloseAfterAllTrashDataDeleteOnce().get(); } catch (Exception e) { Assert.fail(); } From 9cba2abe186ec9c63e2cb0edd007cb370c61bca9 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 13 Jun 2022 13:28:22 +0800 Subject: [PATCH 40/43] enhance unit test. --- .../mledger/ManagedLedgerConfig.java | 10 +- .../bookkeeper/mledger/ManagedTrash.java | 4 +- .../mledger/impl/ManagedLedgerImpl.java | 96 ++++++---- .../mledger/impl/ManagedTrashDisableImpl.java | 6 +- .../mledger/impl/ManagedTrashImpl.java | 67 ++++--- .../mledger/impl/ManagedTrashTest.java | 174 +++++++++++++----- .../pulsar/broker/ServiceConfiguration.java | 4 +- .../pulsar/broker/service/BrokerService.java | 2 +- 8 files changed, 241 insertions(+), 122 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 1f3ba052ee3d2..bf469d7a921d5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -83,7 +83,7 @@ public class ManagedLedgerConfig { private boolean cacheEvictionByMarkDeletedPosition = false; private boolean supportTwoPhaseDeletion = false; private int archiveDataLimitSize = 500; - private int deleteIntervalSeconds = 60; + private int retryDeleteIntervalSeconds = 60; private int maxDeleteCount = 5; public boolean isCreateIfMissing() { @@ -703,12 +703,12 @@ public void setArchiveDataLimitSize(int archiveDataLimitSize) { this.archiveDataLimitSize = archiveDataLimitSize; } - public int getDeleteIntervalSeconds() { - return deleteIntervalSeconds; + public int getRetryDeleteIntervalSeconds() { + return retryDeleteIntervalSeconds; } - public void setDeleteIntervalSeconds(int deleteIntervalSeconds) { - this.deleteIntervalSeconds = deleteIntervalSeconds; + public void setRetryDeleteIntervalSeconds(int retryDeleteIntervalSeconds) { + this.retryDeleteIntervalSeconds = retryDeleteIntervalSeconds; } public void setMaxDeleteCount(int maxDeleteCount) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index 55c61d4954f46..fdcb3a75e6f30 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -65,7 +65,7 @@ enum LedgerType { long getToArchiveDataSize(); - void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx); + CompletableFuture asyncClose(); - CompletableFuture asyncCloseAfterAllTrashDataDeleteOnce(); + CompletableFuture asyncCloseAfterAllLedgerDeleteOnce(); } 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 66c6939e420c5..0f4bbf0215225 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 @@ -1441,22 +1441,7 @@ public synchronized void asyncClose(final CloseCallback callback, final Object c LedgerHandle lh = currentLedger; List> futures = new ArrayList<>(); - - CompletableFuture closeTrashFuture = new CompletableFuture<>(); - futures.add(closeTrashFuture); - - managedTrash.asyncClose(new CloseCallback() { - @Override - public void closeComplete(Object ctx) { - closeTrashFuture.complete(null); - } - - @Override - public void closeFailed(ManagedLedgerException exception, Object ctx) { - closeTrashFuture.completeExceptionally(exception); - } - }, ctx); - + futures.add(managedTrash.asyncClose()); CompletableFuture closeCursorFuture = new CompletableFuture<>(); futures.add(closeCursorFuture); @@ -2902,27 +2887,66 @@ private void deleteAllLedgers(DeleteLedgerCallback callback, Object ctx) { }); return; } - // Mark deletable ledgers - Set deletableLedgers = ledgers.stream().filter(ls -> !ls.getOffloadContext().getBookkeeperDeleted()) - .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); - - // Mark deletable offloaded ledgers - Set deletableOffloadedLedgers = ledgers.stream() - .filter(ls -> ls.getOffloadContext().hasUidMsb()) - .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); - - asyncUpdateTrashData(deletableLedgers, deletableOffloadedLedgers).thenCompose(ignore -> { - managedTrash.triggerDeleteInBackground(); - return deleteMetadata(); - }).thenAccept(ignore -> managedTrash.asyncCloseAfterAllTrashDataDeleteOnce().whenComplete((res, e) -> { - if (e != null) { - log.error("[{}] ManagedTrash all trash data delete once exception.", name, e); + if (managedTrash instanceof ManagedTrashDisableImpl) { + AtomicInteger ledgersToDelete = new AtomicInteger(ledgers.size()); + for (LedgerInfo ls : ledgers) { + if (log.isDebugEnabled()) { + log.debug("[{}] Deleting ledger {}", name, ls); + } + bookKeeper.asyncDeleteLedger(ls.getLedgerId(), (rc, ctx1) -> { + switch (rc) { + case Code.NoSuchLedgerExistsException: + case Code.NoSuchLedgerExistsOnMetadataServerException: + log.warn("[{}] Ledger {} not found when deleting it", name, ls.getLedgerId()); + // Continue anyway + + case BKException.Code.OK: + if (ledgersToDelete.decrementAndGet() == 0) { + // All ledgers deleted, now remove ML metadata + deleteMetadata().whenComplete((res, e) -> { + if (e != null) { + callback.deleteLedgerFailed((ManagedLedgerException) e, ctx); + return; + } + callback.deleteLedgerComplete(ctx); + }); } + break; + + default: + // Handle error + log.warn("[{}] Failed to delete ledger {} -- {}", name, ls.getLedgerId(), + BKException.getMessage(rc)); + int toDelete = ledgersToDelete.get(); + if (toDelete != -1 && ledgersToDelete.compareAndSet(toDelete, -1)) { + // Trigger callback only once + callback.deleteLedgerFailed(createManagedLedgerException(rc), ctx); + } + } + }, null); } - callback.deleteLedgerComplete(ctx); - })).exceptionally(e -> { - callback.deleteLedgerFailed(createManagedLedgerException(e), ctx); - return null; - }); + } else { + // Mark deletable ledgers + Set deletableLedgers = ledgers.stream().filter(ls -> !ls.getOffloadContext().getBookkeeperDeleted()) + .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); + + // Mark deletable offloaded ledgers + Set deletableOffloadedLedgers = ledgers.stream() + .filter(ls -> ls.getOffloadContext().hasUidMsb()) + .map(LedgerInfo::getLedgerId).collect(Collectors.toSet()); + + asyncUpdateTrashData(deletableLedgers, deletableOffloadedLedgers).thenCompose(ignore -> { + managedTrash.triggerDeleteInBackground(); + return deleteMetadata(); + }).thenAccept(ignore -> managedTrash.asyncCloseAfterAllLedgerDeleteOnce().whenComplete((res, e) -> { + if (e != null) { + log.error("[{}] ManagedTrash all trash data delete once exception.", name, e); + } + callback.deleteLedgerComplete(ctx); + })).exceptionally(e -> { + callback.deleteLedgerFailed(createManagedLedgerException(e), ctx); + return null; + }); + } } private CompletableFuture deleteMetadata() { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index 25e73d70b396d..ff50dbe7092c3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -75,12 +75,12 @@ public long getToArchiveDataSize() { } @Override - public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { - callback.closeComplete(ctx); + public CompletableFuture asyncClose() { + return COMPLETABLE_FUTURE; } @Override - public CompletableFuture asyncCloseAfterAllTrashDataDeleteOnce() { + public CompletableFuture asyncCloseAfterAllLedgerDeleteOnce() { return COMPLETABLE_FUTURE; } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 3a70c525f17e5..69a54dba5c671 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -39,7 +39,6 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.common.util.OrderedScheduler; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedTrash; @@ -70,7 +69,7 @@ public class ManagedTrashImpl implements ManagedTrash { private static final String ARCHIVE_SUFFIX = "/" + ARCHIVE; - private static final String TRASH_KEY_SEPARATOR = "-"; + private static final String TRASH_KEY_SEPARATOR = ";"; private static final long EMPTY_LEDGER_ID = -1L; @@ -113,7 +112,9 @@ public class ManagedTrashImpl implements ManagedTrash { private final int archiveDataLimitSize; - private final long deleteIntervalMillis; + private final long retryDeleteIntervalMillis; + + private final long nextDeleteDelayMillis; private final int maxDeleteCount; @@ -137,12 +138,17 @@ public ManagedTrashImpl(ManagedType type, String name, MetadataStore metadataSto this.executor = executor; this.bookKeeper = bookKeeper; this.archiveDataLimitSize = config.getArchiveDataLimitSize(); - this.deleteIntervalMillis = TimeUnit.SECONDS.toMillis(config.getDeleteIntervalSeconds()); + this.retryDeleteIntervalMillis = TimeUnit.SECONDS.toMillis(config.getRetryDeleteIntervalSeconds()); + this.nextDeleteDelayMillis = calculateNextDeleteDelayMillis(this.retryDeleteIntervalMillis); this.maxDeleteCount = Integer.max(1, config.getMaxDeleteCount()); this.managedTrashMXBean = new ManagedTrashMXBeanImpl(this); STATE_UPDATER.set(this, State.None); } + protected long calculateNextDeleteDelayMillis(long retryDeleteIntervalMillis) { + return retryDeleteIntervalMillis / 5; + } + @Override public String name() { return name + "-" + type; @@ -352,24 +358,20 @@ private void triggerDelete(LedgerType type) { } @Override - public CompletableFuture asyncCloseAfterAllTrashDataDeleteOnce() { + public CompletableFuture asyncCloseAfterAllLedgerDeleteOnce() { //ensure can't add more trashData. STATE_UPDATER.set(this, State.FENCED); CompletableFuture future = new CompletableFuture<>(); allTrashDataDeleteOnce(future); return future.thenCompose(ignore -> { CompletableFuture finalFuture = new CompletableFuture<>(); - asyncClose(new AsyncCallbacks.CloseCallback() { - @Override - public void closeComplete(Object ctx) { - finalFuture.complete(null); - } - - @Override - public void closeFailed(ManagedLedgerException exception, Object ctx) { - finalFuture.completeExceptionally(exception); + asyncClose().whenComplete((res, e) -> { + if (e != null) { + finalFuture.completeExceptionally(e); + return; } - }, null); + finalFuture.complete(null); + }); return finalFuture; }); } @@ -448,10 +450,11 @@ public long getToArchiveDataSize() { } @Override - public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { + public CompletableFuture asyncClose() { + CompletableFuture future = new CompletableFuture<>(); if (State.Closed == STATE_UPDATER.get(this)) { - callback.closeComplete(ctx); - return; + future.complete(null); + return future; } if (checkTrashPersistTask != null) { checkTrashPersistTask.cancel(true); @@ -459,12 +462,13 @@ public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { } asyncUpdateTrashData().whenComplete((res, e) -> { if (e != null) { - callback.closeFailed((ManagedLedgerException) e, ctx); + future.completeExceptionally(e); return; } - callback.closeComplete(ctx); + future.complete(null); }); STATE_UPDATER.set(this, State.Closed); + return future; } private CompletableFuture increaseArchiveCountWhenDeleteFailed() { @@ -520,7 +524,7 @@ protected Tuple getToDeleteData(LedgerType type) { } //1.filter trashData which last time and current time differ by no more than deleteIntervalMillis. //2.filter trashData which still exist in managedLedgers - if (System.currentTimeMillis() - entry.getKey().lastDeleteTs < deleteIntervalMillis + if (System.currentTimeMillis() - entry.getKey().lastDeleteTs < retryDeleteIntervalMillis || managedLedgers.containsKey(entry.getKey().getLedgerId())) { filtered = true; continue; @@ -683,7 +687,7 @@ private void continueDeleteIfNecessary() { triggerDeleteInBackground(); continueDeleteImmediately.decrementAndGet(); } else { - scheduledExecutor.schedule(this::triggerDeleteInBackground, deleteIntervalMillis / 5, + scheduledExecutor.schedule(this::triggerDeleteInBackground, nextDeleteDelayMillis, TimeUnit.MILLISECONDS); } } @@ -864,13 +868,18 @@ public int compareTo(TrashKey other) { @Override public String toString() { - return "TrashKey{" + - "retryCount=" + retryCount + - ", ledgerId=" + ledgerId + - ", msb=" + msb + - ", type=" + type + - ", lastDeleteTs=" + lastDeleteTs + - '}'; + return "TrashKey{" + + "retryCount=" + + retryCount + + ", ledgerId=" + + ledgerId + + ", msb=" + + msb + + ", type=" + + type + + ", lastDeleteTs=" + + lastDeleteTs + + '}'; } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java index 6e8eed3ffa499..2318595a008ea 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedTrashTest.java @@ -34,6 +34,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.PulsarMockBookKeeper; @@ -85,9 +86,14 @@ public CompletableFuture put(String path, byte[] value, Optional exp @Override public void asyncDeleteLedger(long lId, AsyncCallback.DeleteCallback cb, Object ctx) { getProgrammedFailure().thenComposeAsync((res) -> { - if (lId >= 10000) { + if (lId >= 10000 && lId < 20000) { throw new IllegalArgumentException("LedgerId is invalid"); } + if (lId >= 20000) { + //if ledgerId >= 20000, whatever ledgers hold it, delete succeed. + deletedLedgers.add(lId); + return FutureUtils.value(null); + } if (ledgers.containsKey(lId)) { ledgers.remove(lId); deletedLedgers.add(lId); @@ -126,7 +132,7 @@ private LedgerInfo buildOffloadLedgerInfo(long ledgerId, UUID uuid) { public void testTrashKeyOrder() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setSupportTwoPhaseDeletion(true); - managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setRetryDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(10); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); @@ -281,16 +287,13 @@ public void testGetToDeleteDataFilterByLastTimeStamp() throws Exception { managedLedgerConfig.setMaxEntriesPerLedger(10); managedLedgerConfig.setRetentionTime(3, TimeUnit.SECONDS); managedLedgerConfig.setMaxDeleteCount(maxDeleteCount); - managedLedgerConfig.setDeleteIntervalSeconds(deleteIntervalSeconds); + managedLedgerConfig.setRetryDeleteIntervalSeconds(deleteIntervalSeconds); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); field.setAccessible(true); ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); - Field field1 = ManagedTrashImpl.class.getDeclaredField("managedLedgers"); - field1.setAccessible(true); - for (int i = 0; i < 30; i++) { long ledgerId = 10000 + i; if (i % 2 == 0) { @@ -357,40 +360,73 @@ public void testGetToDeleteDataFilterByLastTimeStamp() throws Exception { } @Test - public void testRecover() throws Exception { - int entryCount = 30; - ConcurrentSkipListMap map = new ConcurrentSkipListMap<>(); - LedgerInfo emptyLedgerInfo = LedgerInfo.newBuilder().setLedgerId(-1).build(); - for (int i = 0; i < entryCount; i++) { - ManagedTrashImpl.TrashKey trashKey = new ManagedTrashImpl.TrashKey(3, i, 0, ManagedTrash.LedgerType.LEDGER); - map.put(trashKey, emptyLedgerInfo); - } - metadataStore.put("/managed-trash/managed-ledger/my_test_ledger/delete", ManagedTrashImpl.serialize(map), - Optional.of(-1L)); + public void testTriggerDeleteOnTriggerSelf() throws Exception { + int deleteIntervalSeconds = 10; ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setSupportTwoPhaseDeletion(true); managedLedgerConfig.setMaxEntriesPerLedger(10); - managedLedgerConfig.setRetentionTime(1, TimeUnit.SECONDS); + managedLedgerConfig.setRetentionTime(3, TimeUnit.SECONDS); + managedLedgerConfig.setRetryDeleteIntervalSeconds(deleteIntervalSeconds); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); field.setAccessible(true); ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); - Field field1 = ManagedTrashImpl.class.getDeclaredField("trashData"); - field1.setAccessible(true); - ConcurrentSkipListMap trashData = - (ConcurrentSkipListMap) field1.get(managedTrash); - - assertEquals(trashData.size(), entryCount); + long nextDeleteDelayMillis = managedTrash.calculateNextDeleteDelayMillis(deleteIntervalSeconds * 1000); - for (int i = 0; i < entryCount; i++) { - Map.Entry entry1 = map.pollFirstEntry(); - Map.Entry entry2 = trashData.pollFirstEntry(); - assertEquals(entry1.getKey().getLedgerId(), entry2.getKey().getLedgerId()); + for (int i = 0; i < 30; i++) { + managedTrash.appendLedgerTrashData(20000 + i, null, ManagedTrash.LedgerType.LEDGER); } + //trigger delete, the second delete will execute delay `nextDeleteDelayMillis` after first delete finish. + managedTrash.triggerDeleteInBackground(); + + AtomicLong firstTimeDelete = new AtomicLong(); + Awaitility.await().untilAsserted(() -> { + assertTrue(deletedLedgers.size() >= 3); + firstTimeDelete.set(System.currentTimeMillis()); + }); + + Awaitility.await().untilAsserted(() -> { + assertTrue(deletedLedgers.size() >= 6); + assertTrue(System.currentTimeMillis() - firstTimeDelete.get() > nextDeleteDelayMillis); + }); } + @Test + public void testTriggerDeleteOnImmediately() throws Exception { + int deleteIntervalSeconds = 10; + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setMaxEntriesPerLedger(10); + managedLedgerConfig.setRetentionTime(3, TimeUnit.SECONDS); + managedLedgerConfig.setRetryDeleteIntervalSeconds(deleteIntervalSeconds); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + long nextDeleteDelayMillis = managedTrash.calculateNextDeleteDelayMillis(deleteIntervalSeconds * 1000); + + for (int i = 0; i < 30; i++) { + managedTrash.appendLedgerTrashData(20000 + i, null, ManagedTrash.LedgerType.LEDGER); + } + //trigger delete twice, the second delete will continue immediately after first delete finish. + managedTrash.triggerDeleteInBackground(); + managedTrash.triggerDeleteInBackground(); + + AtomicLong firstTimeDelete = new AtomicLong(); + Awaitility.await().untilAsserted(() -> { + assertTrue(deletedLedgers.size() >= 3); + firstTimeDelete.set(System.currentTimeMillis()); + }); + + Awaitility.await().untilAsserted(() -> { + assertTrue(deletedLedgers.size() >= 6); + assertTrue(System.currentTimeMillis() - firstTimeDelete.get() < nextDeleteDelayMillis); + }); + } @Test public void testManagedTrashDelete() throws Exception { @@ -414,7 +450,7 @@ public void testManagedTrashArchive() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setSupportTwoPhaseDeletion(true); - managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setRetryDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); managedLedgerConfig.setMaxDeleteCount(3); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); @@ -424,7 +460,7 @@ public void testManagedTrashArchive() throws Exception { ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); - //the ledgerId >= 10000, it will delete failed. see line_142. + //the ledgerId >= 10000, it will delete failed. see line_89. for (int i = 0; i < entryCount; i++) { managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); } @@ -472,7 +508,7 @@ public void testManagedTrashClose() throws Exception { //when managedTrash close, it will persist trashData. ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setSupportTwoPhaseDeletion(true); - managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setRetryDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(10); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); @@ -533,7 +569,7 @@ public void testGetAllArchiveIndex() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setSupportTwoPhaseDeletion(true); - managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setRetryDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); managedLedgerConfig.setMaxDeleteCount(3); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); @@ -543,7 +579,7 @@ public void testGetAllArchiveIndex() throws Exception { ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); - //the ledgerId >= 10000, it will delete failed. see line_142. + //the ledgerId >= 10000, it will delete failed. see line_89. for (int i = 0; i < entryCount; i++) { managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); } @@ -565,7 +601,7 @@ public void testGetArchiveData() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setSupportTwoPhaseDeletion(true); - managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setRetryDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); managedLedgerConfig.setMaxDeleteCount(3); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); @@ -575,7 +611,7 @@ public void testGetArchiveData() throws Exception { ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); - //the ledgerId >= 10000, it will delete failed. see line_142. + //the ledgerId >= 10000, it will delete failed. see line_89. for (int i = 0; i < entryCount; i++) { managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); } @@ -613,7 +649,7 @@ public void testAsyncCloseAfterAllTrashDataDeleteOnce() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setSupportTwoPhaseDeletion(true); - managedLedgerConfig.setDeleteIntervalSeconds(1); + managedLedgerConfig.setRetryDeleteIntervalSeconds(1); managedLedgerConfig.setArchiveDataLimitSize(archiveLimit); managedLedgerConfig.setMaxDeleteCount(maxDeleteCount); ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); @@ -623,13 +659,22 @@ public void testAsyncCloseAfterAllTrashDataDeleteOnce() throws Exception { ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); - //the ledgerId >= 10000, it will delete failed. see line_142. + //the ledgerId >= 10000, it will delete failed. see line_89. for (int i = 0; i < entryCount; i++) { - managedTrash.appendLedgerTrashData(10000 + i, null, ManagedTrash.LedgerType.LEDGER); + long ledgerId = 10000 + i; + if (i % 2 == 0) { + managedTrash.appendLedgerTrashData(ledgerId, null, ManagedTrash.LedgerType.LEDGER); + } else { + //build offload ledger + UUID uuid = UUID.randomUUID(); + LedgerInfo ledgerInfo = buildOffloadLedgerInfo(ledgerId, uuid); + managedTrash.appendLedgerTrashData(ledgerId, ledgerInfo, ManagedTrash.LedgerType.OFFLOAD_LEDGER); + } } try { - managedTrash.asyncCloseAfterAllTrashDataDeleteOnce().get(); + //all ledger will least delete once. + managedTrash.asyncCloseAfterAllLedgerDeleteOnce().get(); } catch (Exception e) { Assert.fail(); } @@ -642,12 +687,18 @@ public void testAsyncCloseAfterAllTrashDataDeleteOnce() throws Exception { assertEquals(trashData.size(), 30); - Optional> lastLedgerTrashData = - trashData.descendingMap().entrySet().stream() - .filter(ele -> ele.getKey().getType() != ManagedTrash.LedgerType.OFFLOAD_LEDGER) - .findFirst(); - assertTrue(lastLedgerTrashData.isPresent()); - assertTrue(lastLedgerTrashData.get().getKey().getRetryCount() < maxDeleteCount); + int offloadNotDeleteCount = 0; + for (Map.Entry entry : trashData.entrySet()) { + if (ManagedTrash.LedgerType.LEDGER == entry.getKey().getType()) { + assertTrue(entry.getKey().getRetryCount() < maxDeleteCount); + } + if (ManagedTrash.LedgerType.OFFLOAD_LEDGER == entry.getKey().getType()) { + assertTrue(entry.getKey().getRetryCount() == maxDeleteCount); + offloadNotDeleteCount++; + } + } + + assertEquals(offloadNotDeleteCount, 15); byte[] content = persistedData.get("/managed-trash/managed-ledger/my_test_ledger/delete"); @@ -661,4 +712,39 @@ public void testAsyncCloseAfterAllTrashDataDeleteOnce() throws Exception { assertEquals(entry1.getKey().compareTo(entry2.getKey()), 0); } } + + @Test + public void testRecover() throws Exception { + int entryCount = 30; + ConcurrentSkipListMap map = new ConcurrentSkipListMap<>(); + LedgerInfo emptyLedgerInfo = LedgerInfo.newBuilder().setLedgerId(-1).build(); + for (int i = 0; i < entryCount; i++) { + ManagedTrashImpl.TrashKey trashKey = new ManagedTrashImpl.TrashKey(3, i, 0, ManagedTrash.LedgerType.LEDGER); + map.put(trashKey, emptyLedgerInfo); + } + metadataStore.put("/managed-trash/managed-ledger/my_test_ledger/delete", ManagedTrashImpl.serialize(map), + Optional.of(-1L)); + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setSupportTwoPhaseDeletion(true); + managedLedgerConfig.setMaxEntriesPerLedger(10); + managedLedgerConfig.setRetentionTime(1, TimeUnit.SECONDS); + ManagedLedger ledger = factory.open("my_test_ledger", managedLedgerConfig); + + Field field = ManagedLedgerImpl.class.getDeclaredField("managedTrash"); + field.setAccessible(true); + ManagedTrashImpl managedTrash = (ManagedTrashImpl) field.get(ledger); + + Field field1 = ManagedTrashImpl.class.getDeclaredField("trashData"); + field1.setAccessible(true); + ConcurrentSkipListMap trashData = + (ConcurrentSkipListMap) field1.get(managedTrash); + + assertEquals(trashData.size(), entryCount); + + for (int i = 0; i < entryCount; i++) { + Map.Entry entry1 = map.pollFirstEntry(); + Map.Entry entry2 = trashData.pollFirstEntry(); + assertEquals(entry1.getKey().getLedgerId(), entry2.getKey().getLedgerId()); + } + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index d6064e4a4d5ab..f1de1197072e7 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2532,9 +2532,9 @@ public class ServiceConfiguration implements PulsarConfiguration { @FieldContext( dynamic = true, category = CATEGORY_STORAGE_ML, - doc = "The interval of deletion at same ledger in managedTrash. (Default 60 seconds)" + doc = "The interval of retry delete same ledger again. (Default 60 seconds)" ) - private int managedTrashDeleteInternalSeconds = 60; + private int managedTrashRetryDeleteInternalSeconds = 60; @FieldContext( dynamic = true, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 8d0e79787ef65..45d1f09a629c3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1593,7 +1593,7 @@ public CompletableFuture getManagedLedgerConfig(TopicName t managedLedgerConfig.setSupportTwoPhaseDeletion( serviceConfig.isManagedLedgerSupportTwoPhaseDeletion()); managedLedgerConfig.setArchiveDataLimitSize(serviceConfig.getManagedTrashArchiveDataLimitSize()); - managedLedgerConfig.setDeleteIntervalSeconds(serviceConfig.getManagedTrashDeleteInternalSeconds()); + managedLedgerConfig.setRetryDeleteIntervalSeconds(serviceConfig.getManagedTrashRetryDeleteInternalSeconds()); managedLedgerConfig.setMaxDeleteCount(serviceConfig.getManagedLedgerMaxDeleteCount()); OffloadPoliciesImpl nsLevelOffloadPolicies = From 8fa183ddf5aec545fed72c6fce00dbcfa2c909b8 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 13 Jun 2022 13:52:52 +0800 Subject: [PATCH 41/43] Add api doc info. --- .../bookkeeper/mledger/ManagedTrash.java | 43 +++++++++++++++++-- .../mledger/impl/ManagedTrashDisableImpl.java | 11 ----- .../mledger/impl/ManagedTrashImpl.java | 2 - .../mledger/impl/ManagedTrashMXBeanImpl.java | 10 ++++- 4 files changed, 47 insertions(+), 19 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java index fdcb3a75e6f30..18a1e3a666ca5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedTrash.java @@ -47,25 +47,60 @@ enum LedgerType { LEDGER } + /** + * ManagedTrash name. + * + * @return full topic name + type + */ String name(); + /** + * Initialize. + */ CompletableFuture initialize(); + /** + * Append waiting to delete ledger. + * + * @param ledgerId ledgerId + * @param context ledgerInfo, if offload ledger, need offload context + * @param type LEDGER or OFFLOAD_LEDGER + * @throws ManagedLedgerException + */ void appendLedgerTrashData(long ledgerId, LedgerInfo context, LedgerType type) throws ManagedLedgerException; + /** + * Persist trash data to meta store. + */ CompletableFuture asyncUpdateTrashData(); + /** + * Trigger deletion procedure. + */ void triggerDeleteInBackground(); + /** + * Get all archive index, it needs combine with getArchiveData. + */ CompletableFuture> getAllArchiveIndex(); + /** + * Get archive data detail info. + * + * @param index archive index + * @return + */ CompletableFuture> getArchiveData(long index); - long getTrashDataSize(); - - long getToArchiveDataSize(); - + /** + * Async close managedTrash, it will persist trash data to meta store. + * @return + */ CompletableFuture asyncClose(); + /** + * Async close managedTrash, it can ensure that all ledger least delete once (exclude offload_ledger). + * @return + */ CompletableFuture asyncCloseAfterAllLedgerDeleteOnce(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java index ff50dbe7092c3..734a88e888c3a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashDisableImpl.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedTrash; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; @@ -64,16 +63,6 @@ public CompletableFuture> getArchiveD return (CompletableFuture>) COMPLETABLE_FUTURE; } - @Override - public long getTrashDataSize() { - return 0; - } - - @Override - public long getToArchiveDataSize() { - return 0; - } - @Override public CompletableFuture asyncClose() { return COMPLETABLE_FUTURE; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java index 69a54dba5c671..55e835378917b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashImpl.java @@ -439,12 +439,10 @@ public CompletableFuture> getArchiveData(final long in }, executor.chooseThread(name)); } - @Override public long getTrashDataSize() { return trashData.size(); } - @Override public long getToArchiveDataSize() { return toArchiveCount.get(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java index 625876252af39..0afb33112f10a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedTrashMXBeanImpl.java @@ -41,7 +41,10 @@ public String getName() { @Override public long getCurrentNumberOfLedgersWaitingToDelete() { - return managedTrash.getTrashDataSize(); + if (managedTrash instanceof ManagedTrashImpl) { + return ((ManagedTrashImpl) managedTrash).getTrashDataSize(); + } + return 0; } @Override @@ -56,7 +59,10 @@ public long getTotalNumberOfDeleteLedgers() { @Override public long getCurrentNumberOfLedgersWaitingToArchive() { - return managedTrash.getToArchiveDataSize(); + if (managedTrash instanceof ManagedTrashImpl) { + return ((ManagedTrashImpl) managedTrash).getToArchiveDataSize(); + } + return 0; } @Override From 91d8fcc28ed8d41bfb2583eea7b8b9063499ba84 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 13 Jun 2022 13:59:08 +0800 Subject: [PATCH 42/43] code clean. --- .../org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 3 ++- 1 file changed, 2 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 0f4bbf0215225..15c0b25f0e59b 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 @@ -2909,7 +2909,8 @@ private void deleteAllLedgers(DeleteLedgerCallback callback, Object ctx) { return; } callback.deleteLedgerComplete(ctx); - }); } + }); + } break; default: From 793687c82de507def4582237175787499067a6c9 Mon Sep 17 00:00:00 2001 From: horizonzy Date: Mon, 13 Jun 2022 14:33:48 +0800 Subject: [PATCH 43/43] fix code style. --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 45d1f09a629c3..954ec07a86e39 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1593,7 +1593,8 @@ public CompletableFuture getManagedLedgerConfig(TopicName t managedLedgerConfig.setSupportTwoPhaseDeletion( serviceConfig.isManagedLedgerSupportTwoPhaseDeletion()); managedLedgerConfig.setArchiveDataLimitSize(serviceConfig.getManagedTrashArchiveDataLimitSize()); - managedLedgerConfig.setRetryDeleteIntervalSeconds(serviceConfig.getManagedTrashRetryDeleteInternalSeconds()); + managedLedgerConfig.setRetryDeleteIntervalSeconds( + serviceConfig.getManagedTrashRetryDeleteInternalSeconds()); managedLedgerConfig.setMaxDeleteCount(serviceConfig.getManagedLedgerMaxDeleteCount()); OffloadPoliciesImpl nsLevelOffloadPolicies =