From 1adcedc70905b74ea0feb5c4e335afa8915b7687 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Wed, 19 May 2021 10:28:15 +0800 Subject: [PATCH 01/18] Add ack --- conf/broker.conf | 7 + .../mledger/ManagedLedgerConfig.java | 20 + .../mledger/impl/LongPairRangeSetWrapper.java | 146 ++++++ .../mledger/impl/ManagedCursorImpl.java | 417 +++++++++++++++--- .../src/main/proto/MLDataFormats.proto | 9 + .../mledger/impl/ManagedCursorTest.java | 6 +- .../mledger/impl/ManagedLedgerTest.java | 3 +- .../pulsar/broker/ServiceConfiguration.java | 11 + .../pulsar/broker/service/BrokerService.java | 2 + 9 files changed, 543 insertions(+), 78 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LongPairRangeSetWrapper.java diff --git a/conf/broker.conf b/conf/broker.conf index ce94927429d1c..73350e2b88c1b 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -918,6 +918,13 @@ managedLedgerMaxUnackedRangesToPersist=10000 # zookeeper. managedLedgerMaxUnackedRangesToPersistInZooKeeper=1000 +# If enabled, the maximum "acknowledgment holes" will not be limited. +# Only part of hotspot data will keep in memory by using LRU. +enableLruCacheMaxUnackedRanges=true + +# Max Bytes of UnackedRanges can keep in memory. +maxUnackedRangesInMemoryBytes=3145728 + # Skip reading non-recoverable/unreadable data-ledger under managed-ledger's list. It helps when data-ledgers gets # corrupted at bookkeeper and managed-cursor is stuck at that ledger. autoSkipNonRecoverableData=false 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 2f4e098d677e5..e40b656b48679 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 @@ -46,6 +46,8 @@ public class ManagedLedgerConfig { private boolean createIfMissing = true; private int maxUnackedRangesToPersist = 10000; private int maxBatchDeletedIndexToPersist = 10000; + private boolean enableLruCacheMaxUnackedRanges = false; + private long maxUnackedRangesInMemoryBytes = 3145728; private boolean deletionAtBatchIndexLevelEnabled = true; private int maxUnackedRangesToPersistInZk = 1000; private int maxEntriesPerLedger = 50000; @@ -488,6 +490,24 @@ public void setMaxUnackedRangesToPersistInZk(int maxUnackedRangesToPersistInZk) this.maxUnackedRangesToPersistInZk = maxUnackedRangesToPersistInZk; } + public boolean isEnableLruCacheMaxUnackedRanges() { + return enableLruCacheMaxUnackedRanges; + } + + public ManagedLedgerConfig setEnableLruCacheMaxUnackedRanges(boolean enableLruCacheMaxUnackedRanges) { + this.enableLruCacheMaxUnackedRanges = enableLruCacheMaxUnackedRanges; + return this; + } + + public long getMaxUnackedRangesInMemoryBytes() { + return maxUnackedRangesInMemoryBytes; + } + + public ManagedLedgerConfig setMaxUnackedRangesInMemoryBytes(long maxUnackedRangesInMemoryBytes) { + this.maxUnackedRangesInMemoryBytes = maxUnackedRangesInMemoryBytes; + return this; + } + /** * Get ledger offloader which will be used to offload ledgers to longterm storage. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LongPairRangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LongPairRangeSetWrapper.java new file mode 100644 index 0000000000000..ef2953f210488 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LongPairRangeSetWrapper.java @@ -0,0 +1,146 @@ +/** + * 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 com.google.common.collect.Range; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; +import org.apache.pulsar.common.util.collections.LongPairRangeSet; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicInteger; + +public class LongPairRangeSetWrapper> implements LongPairRangeSet { + + private final TreeMap> ledgerIdToPosition = new TreeMap<>(); + final LongPairConsumer positionRangeConverter; + final ManagedLedgerConfig config; + + public LongPairRangeSetWrapper(ManagedLedgerConfig config, LongPairConsumer positionRangeConverter) { + this.positionRangeConverter = positionRangeConverter; + this.config = config; + } + + @Override + public void addOpenClosed(long lowerKey, long lowerValue, long upperKey, long upperValue) { + LongPairRangeSet pairRangeSet = ledgerIdToPosition + .computeIfAbsent(upperKey, (key) -> getLongPairRangeSetImpl()); + if (config.isEnableLruCacheMaxUnackedRanges()) { + // lru switch + } else { + pairRangeSet.addOpenClosed(lowerKey, lowerValue, upperKey, upperValue); + } + } + + private LongPairRangeSet getLongPairRangeSetImpl() { + return config.isUnackedRangesOpenCacheSetEnabled() + ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter) + : new LongPairRangeSet.DefaultRangeSet<>(positionRangeConverter); + } + + @Override + public boolean contains(long key, long value) { + LongPairRangeSet longPairRangeSet = ledgerIdToPosition.get(key); + if (longPairRangeSet == null) { + return false; + } + return longPairRangeSet.contains(key, value); + } + + @Override + public Range rangeContaining(long key, long value) { + LongPairRangeSet longPairRangeSet = ledgerIdToPosition.get(key); + if (longPairRangeSet == null) { + return null; + } + return longPairRangeSet.rangeContaining(key, value); + } + + @Override + public void removeAtMost(long key, long value) { + LongPairRangeSet longPairRangeSet = ledgerIdToPosition.get(key); + if (longPairRangeSet == null) { + return; + } + longPairRangeSet.removeAtMost(key, value); + } + + @Override + public boolean isEmpty() { + return ledgerIdToPosition.isEmpty(); + } + + @Override + public void clear() { + ledgerIdToPosition.clear(); + } + + @Override + public Range span() { + Range rangeFirst = ledgerIdToPosition.firstEntry().getValue().span(); + Range rangeLast = ledgerIdToPosition.lastEntry().getValue().span(); + return Range.openClosed(rangeFirst.lowerEndpoint(), rangeLast.upperEndpoint()); + } + + @Override + public List> asRanges() { + List> ranges = new ArrayList<>(); + ledgerIdToPosition.values().forEach(ledgerIdToPosition -> ranges.addAll(ledgerIdToPosition.asRanges())); + return ranges; + } + + @Override + public void forEach(RangeProcessor action) { + ledgerIdToPosition.values().forEach(ledgerIdToPosition -> ledgerIdToPosition.forEach(action)); + } + + @Override + public void forEach(RangeProcessor action, LongPairConsumer consumer) { + ledgerIdToPosition.values().forEach(ledgerIdToPosition -> ledgerIdToPosition.forEach(action, consumer)); + } + + @Override + public int size() { + AtomicInteger size = new AtomicInteger(0); + ledgerIdToPosition.values().forEach(ledgerIdToPosition -> size.addAndGet(ledgerIdToPosition.size())); + return size.get(); + } + + @Override + public Range firstRange() { + Map.Entry> entry = ledgerIdToPosition.firstEntry(); + if (entry == null) { + return null; + } + return entry.getValue().firstRange(); + } + + @Override + public Range lastRange() { + Map.Entry> entry = ledgerIdToPosition.lastEntry(); + if (entry == null) { + return null; + } + return entry.getValue().lastRange(); + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index dc89dbb1a56ca..5fc9417bb9746 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -45,11 +45,15 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -62,9 +66,9 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; -import org.apache.bookkeeper.client.AsyncCallback.DeleteCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; @@ -96,7 +100,9 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; +import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet; @@ -133,6 +139,8 @@ public class ManagedCursorImpl implements ManagedCursor { public static final int FALSE = 0; public static final int TRUE = 1; + public static final String LRU_ENTRY = "lru-entry"; + public static final String LRU_MARKER = "lru-marker"; private static final AtomicIntegerFieldUpdater RESET_CURSOR_IN_PROGRESS_UPDATER = AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "resetCursorInProgress"); @SuppressWarnings("unused") @@ -158,6 +166,9 @@ public class ManagedCursorImpl implements ManagedCursor { // Stat of the cursor z-node private volatile Stat cursorLedgerStat; + // The key is the ledger we are acknowledging, value is the position that saves the ack information. + private Map rangeMarker = new ConcurrentHashMap<>(); + private static final LongPairConsumer positionRangeConverter = PositionImpl::new; private static final LongPairConsumer recyclePositionRangeConverter = (key, value) -> { PositionImplRecyclable position = PositionImplRecyclable.create(); @@ -515,7 +526,7 @@ void initialize(PositionImpl position, Map properties, final VoidC ledger.getName(), name, messagesConsumedCounter, markDeletePosition, readPosition); } - createNewMetadataLedger(new VoidCallback() { + createNewMetadataLedgerAndSwitch(new VoidCallback() { @Override public void operationComplete() { STATE_UPDATER.set(ManagedCursorImpl.this, State.Open); @@ -2304,12 +2315,12 @@ void startCreatingNewMetadataLedger() { // Check if we can immediately switch to a new metadata ledger if (PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER.get(this) == 0) { - createNewMetadataLedger(); + createNewMetadataLedgerAndSwitch(); } } - void createNewMetadataLedger() { - createNewMetadataLedger(new VoidCallback() { + void createNewMetadataLedgerAndSwitch() { + createNewMetadataLedgerAndSwitch(new VoidCallback() { @Override public void operationComplete() { // We now have a new ledger where we can write @@ -2352,9 +2363,77 @@ void internalFlushPendingMarkDeletes() { internalMarkDelete(lastEntry); } - void createNewMetadataLedger(final VoidCallback callback) { + void createNewMetadataLedgerAndSwitch(final VoidCallback callback) { ledger.mbean.startCursorLedgerCreateOp(); + doCreateNewMetadataLedger().thenAccept(newLedgerHandle -> { + ledger.mbean.endCursorLedgerCreateOp(); + MarkDeleteEntry mdEntry = lastMarkDeleteEntry; + if (config.isEnableLruCacheMaxUnackedRanges()) { + // copy all available entry to new ledger + copyLruEntriesToNewLedger(cursorLedger, newLedgerHandle).whenComplete((res, e) -> { + if (e != null) { + deleteLedger(newLedgerHandle); + callback.operationFailed(createManagedLedgerException(e)); + } else { + callback.operationComplete(); + } + }); + return; + } + // Created the ledger, now write the last position + // content + persistPositionToLedger(newLedgerHandle, mdEntry, new VoidCallback() { + @Override + public void operationComplete() { + if (log.isDebugEnabled()) { + log.debug("[{}] Persisted position {} for cursor {}", ledger.getName(), + mdEntry.newPosition, name); + } + switchToNewLedger(newLedgerHandle, new VoidCallback() { + @Override + public void operationComplete() { + callback.operationComplete(); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + // it means it failed to switch the newly created ledger so, it should be + // deleted to prevent leak + deleteLedger(newLedgerHandle); + callback.operationFailed(exception); + } + }); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + log.warn("[{}] Failed to persist position {} for cursor {}", ledger.getName(), + mdEntry.newPosition, name); + + deleteLedger(newLedgerHandle); + callback.operationFailed(exception); + } + }); + }).exceptionally(e -> { + ledger.mbean.endCursorLedgerCreateOp(); + return null; + }); + + } + + private void deleteLedger(LedgerHandle ledgerHandle) { + ledger.mbean.startCursorLedgerDeleteOp(); + bookkeeper.asyncDeleteLedger(ledgerHandle.getId(), (int rc, Object ctx) -> { + ledger.mbean.endCursorLedgerDeleteOp(); + if (rc != BKException.Code.OK) { + log.warn("[{}] Failed to delete orphan ledger {}", ledger.getName(), + ledgerHandle.getId()); + } + }, null); + } + private CompletableFuture doCreateNewMetadataLedger() { + CompletableFuture future = new CompletableFuture<>(); ledger.asyncCreateLedger(bookkeeper, config, digestType, (rc, lh, ctx) -> { if (ledger.checkAndCompleteLedgerOpTask(rc, lh, ctx)) { @@ -2366,61 +2445,18 @@ void createNewMetadataLedger(final VoidCallback callback) { if (rc != BKException.Code.OK) { log.warn("[{}] Error creating ledger for cursor {}: {}", ledger.getName(), name, BKException.getMessage(rc)); - callback.operationFailed(new ManagedLedgerException(BKException.getMessage(rc))); + future.completeExceptionally(new ManagedLedgerException(BKException.getMessage(rc))); return; } if (log.isDebugEnabled()) { log.debug("[{}] Created ledger {} for cursor {}", ledger.getName(), lh.getId(), name); } - // Created the ledger, now write the last position - // content - MarkDeleteEntry mdEntry = lastMarkDeleteEntry; - persistPositionToLedger(lh, mdEntry, new VoidCallback() { - @Override - public void operationComplete() { - if (log.isDebugEnabled()) { - log.debug("[{}] Persisted position {} for cursor {}", ledger.getName(), - mdEntry.newPosition, name); - } - switchToNewLedger(lh, new VoidCallback() { - @Override - public void operationComplete() { - callback.operationComplete(); - } - - @Override - public void operationFailed(ManagedLedgerException exception) { - // it means it failed to switch the newly created ledger so, it should be - // deleted to prevent leak - bookkeeper.asyncDeleteLedger(lh.getId(), (int rc, Object ctx) -> { - if (rc != BKException.Code.OK) { - log.warn("[{}] Failed to delete orphan ledger {}", ledger.getName(), - lh.getId()); - } - }, null); - callback.operationFailed(exception); - } - }); - } - - @Override - public void operationFailed(ManagedLedgerException exception) { - log.warn("[{}] Failed to persist position {} for cursor {}", ledger.getName(), - mdEntry.newPosition, name); - - ledger.mbean.startCursorLedgerDeleteOp(); - bookkeeper.asyncDeleteLedger(lh.getId(), new DeleteCallback() { - @Override - public void deleteComplete(int rc, Object ctx) { - ledger.mbean.endCursorLedgerDeleteOp(); - } - }, null); - callback.operationFailed(exception); - } - }); + future.complete(lh); })); }, LedgerMetadataUtils.buildAdditionalMetadataForCursor(name)); + + return future; } private List buildPropertiesMap(Map properties) { @@ -2437,6 +2473,37 @@ private List buildPropertiesMap(Map properties) { return longProperties; } + private MLDataFormats.NestedPositionInfo buildPositionInfo(long ledgerId, long entryId) { + return MLDataFormats.NestedPositionInfo.newBuilder() + .setLedgerId(ledgerId).setEntryId(entryId).build(); + } + + private List buildMarkerIndexMap( + Map rangeMarker) { + MLDataFormats.MarkerIndexInfo.Builder builder = MLDataFormats.MarkerIndexInfo.newBuilder(); + return rangeMarker.entrySet().stream().map((entry) -> + builder.setTargetLedgerId(entry.getKey()).addEntryPosition(entry.getValue()).build()) + .collect(Collectors.toList()); + } + + private Map buildPropertiesMap(List longProperties) { + Map propertiesMap = Maps.newHashMapWithExpectedSize(longProperties.size()); + if (CollectionUtils.isEmpty(longProperties)) { + return propertiesMap; + } + longProperties.forEach((longProperty) -> + propertiesMap.put(longProperty.getName(), longProperty.getValue())); + return propertiesMap; + } + + private LongProperty createLruEntryTag() { + return LongProperty.newBuilder().setName(LRU_ENTRY).setValue(TRUE).build(); + } + + private LongProperty createLruMarkerTag() { + return LongProperty.newBuilder().setName(LRU_MARKER).setValue(TRUE).build(); + } + private List buildIndividualDeletedMessageRanges() { lock.readLock().lock(); try { @@ -2497,7 +2564,207 @@ private List buildBatchEntryDeletio return result; } - void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { + void persistPositionToLedger(final LedgerHandle newLedgerHandle, MarkDeleteEntry mdEntry, final VoidCallback callback) { + if (config.isEnableLruCacheMaxUnackedRanges()) { + persistPositionToMultiEntry(newLedgerHandle, mdEntry, callback); + } else { + persistPositionToSingleEntry(newLedgerHandle, mdEntry, callback); + } + } + + private void persistPositionToMultiEntry(LedgerHandle lh, MarkDeleteEntry mdEntry, VoidCallback callback) { + checkNotNull(lh); + // get range info and group by ledger ID + Map> rangeGroupByLedgerId = getRangeGroupByLedgerId(); + Map> deletionIndexInfoGroupByLedgerId = + getDeletionIndexInfosGroupByLedgerId(); + List> callbacks = Collections.synchronizedList(new ArrayList<>()); + // build PositionInfo + PositionInfo.Builder entryBuilder = getPositionBuilder(mdEntry).addProperties(createLruEntryTag()); + // save entries to ledger + for (Map.Entry> messageRanges : rangeGroupByLedgerId.entrySet()) { + CompletableFuture future = new CompletableFuture<>(); + callbacks.add(future); + PositionInfo pi = entryBuilder.addAllIndividualDeletedMessages(messageRanges.getValue()) + .addAllBatchedEntryDeletionIndexInfo( + deletionIndexInfoGroupByLedgerId.get(messageRanges.getKey())) + .build(); + lh.asyncAddEntry(pi.toByteArray(), (rc, lh1, entryId, ctx) -> { + if (rc == BKException.Code.OK) { + rangeMarker.put(messageRanges.getKey(), + buildPositionInfo(lh1.getLedgerMetadata().getLedgerId(), entryId)); + mbean.persistToLedger(true); + future.complete(null); + } else { + future.completeExceptionally(BKException.create(rc)); + } + }, null); + } + // save marker + FutureUtil.waitForAll(callbacks).thenCompose((x) -> { + // no entry is saved, so marker is not dirty + if (rangeGroupByLedgerId.size() < 1) { + return CompletableFuture.completedFuture(null); + } + PositionInfo.Builder writerBuilder = getPositionBuilder(mdEntry).addProperties(createLruMarkerTag()); + writerBuilder.addAllMarkerIndexInfo(buildMarkerIndexMap(rangeMarker)); + return saveMarker(lh, writerBuilder.build().toByteArray()); + }).thenCompose((x) -> { + // callback and check whether should create new ledger + return checkIfNeedCreateNewLruLedgerAndSwitch(lh); + }).thenAccept((newLedgerHandler) -> { + callback.operationComplete(); + STATE_UPDATER.set(ManagedCursorImpl.this, State.Open); + mbean.persistToLedger(true); + }).exceptionally(e -> { + log.warn("[{}] Error updating cursor {} position {} in meta-ledger {}", ledger.getName(), name, + mdEntry.newPosition, lh.getId(), e); + STATE_UPDATER.compareAndSet(ManagedCursorImpl.this, State.Open, State.NoLedger); + mbean.persistToLedger(false); + callback.operationFailed(e instanceof ManagedLedgerException ? + (ManagedLedgerException) e : new ManagedLedgerException(e)); + return null; + }); + } + + private PositionInfo.Builder getPositionBuilder(MarkDeleteEntry mdEntry) { + PositionImpl position = mdEntry.newPosition; + return PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) + .addAllProperties(buildPropertiesMap(mdEntry.properties)) + .setEntryId(position.getEntryId()); + } + + private CompletableFuture saveMarker(LedgerHandle lh, byte[] bytes) { + CompletableFuture writeMarker = new CompletableFuture<>(); + lh.asyncAddEntry(bytes, (rc, lh1, entryId, ctx) -> { + if (rc == BKException.Code.OK) { + writeMarker.complete(null); + } else { + writeMarker.completeExceptionally(BKException.create(rc)); + } + }, null); + return writeMarker; + } + + private CompletableFuture checkIfNeedCreateNewLruLedgerAndSwitch(LedgerHandle lh) { + if (!shouldCloseLedger(lh)) { + return CompletableFuture.completedFuture(null); + } + //create new ledger and copy entries + CompletableFuture future = new CompletableFuture<>(); + doCreateNewMetadataLedger().thenCompose((newLedgerHandle) -> { + return copyLruEntriesToNewLedger(lh, newLedgerHandle); + }).exceptionally((exception) -> { + future.completeExceptionally(exception); + return null; + }); + return future; + } + + private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle, LedgerHandle newHandle) { + CompletableFuture result = new CompletableFuture<>(); + // copy entries to new ledger + List> futures = Collections.synchronizedList(new ArrayList<>()); + Map dirtyLedgerRecorder = new ConcurrentHashMap<>(); + try { + for (Map.Entry longPositionEntry : rangeMarker.entrySet()) { + MLDataFormats.NestedPositionInfo position = longPositionEntry.getValue(); + Enumeration entryEnumeration = + oldHandle.readEntries(position.getEntryId(), position.getEntryId()); + if (entryEnumeration == null) { + log.warn("entry is deleted {}:{}", oldHandle.getLedgerMetadata().getLedgerId(), position.getEntryId()); + continue; + } + final CompletableFuture future = new CompletableFuture<>(); + futures.add(future); + LedgerEntry entry = entryEnumeration.nextElement(); + newHandle.asyncAddEntry(entry.getEntry(), (rc, ledgerHandle, entryId, ctx) -> { + if (rc != BKException.Code.OK) { + future.completeExceptionally(BKException.create(rc)); + } else { + dirtyLedgerRecorder.put(longPositionEntry.getKey(), + buildPositionInfo(newHandle.getLedgerMetadata().getLedgerId(), entryId)); + future.complete(null); + } + }, null); + } + } catch (Exception e) { + result.completeExceptionally(e); + return result; + } + // copy marker to new Ledger + FutureUtil.waitForAll(futures).whenComplete((res, e) -> { + if (e != null) { + result.completeExceptionally(e); + return; + } + Map clonedMap = Maps.newHashMap(rangeMarker); + clonedMap.putAll(dirtyLedgerRecorder); + PositionInfo.Builder writerBuilder = getPositionBuilder(lastMarkDeleteEntry) + .addProperties(createLruMarkerTag()).addAllMarkerIndexInfo(buildMarkerIndexMap(clonedMap)); + saveMarker(newHandle, writerBuilder.build().toByteArray()).whenComplete((re, ex) -> { + if (ex != null) { + result.completeExceptionally(ex); + } else { + rangeMarker.putAll(clonedMap); + result.complete(null); + } + }); + }); + return result; + } + + private Optional getLastAvailableMarker(LedgerHandle ledgerHandle) { + long entryId = ledgerHandle.getLastAddConfirmed(); + try { + for (long i = entryId; i >= 0; i--) { + Enumeration entryEnumeration = ledgerHandle.readEntries(i, i); + if (!entryEnumeration.hasMoreElements()) { + return Optional.empty(); + } + LedgerEntry ledgerEntry = entryEnumeration.nextElement(); + PositionInfo positionInfo = PositionInfo.parseFrom(ledgerEntry.getEntry()); + Map propertiesMap = buildPropertiesMap(positionInfo.getPropertiesList()); + if(!propertiesMap.containsKey(LRU_ENTRY) && propertiesMap.containsKey(LRU_MARKER)){ + log.info("Currently not using lru mode"); + return Optional.empty(); + } + if (propertiesMap.containsKey(LRU_MARKER)) { + return Optional.of(positionInfo); + } + } + } catch (Exception e) { + log.error("fail to get last available marker", e); + } + return Optional.empty(); + } + + private Map> getDeletionIndexInfosGroupByLedgerId() { + List batchDeletionIndexInfos + = buildBatchEntryDeletionIndexInfoList(); + Map> ledgerIdToIndexInfo = new HashMap<>(); + batchDeletionIndexInfos.forEach(indexInfo -> { + long ledgerId = indexInfo.getPosition().getLedgerId(); + List list = ledgerIdToIndexInfo + .computeIfAbsent(ledgerId, (le) -> new ArrayList<>()); + list.add(indexInfo); + }); + return ledgerIdToIndexInfo; + } + + private Map> getRangeGroupByLedgerId() { + List rangeList = buildIndividualDeletedMessageRanges(); + Map> ledgerIdToMessageRange = new HashMap<>(); + rangeList.forEach(messageRange -> { + long ledgerId = messageRange.getLowerEndpoint().getLedgerId(); + List list = ledgerIdToMessageRange + .computeIfAbsent(ledgerId, (le) -> new ArrayList<>()); + list.add(messageRange); + }); + return ledgerIdToMessageRange; + } + + private void persistPositionToSingleEntry(LedgerHandle lh, MarkDeleteEntry mdEntry, VoidCallback callback) { PositionImpl position = mdEntry.newPosition; PositionInfo pi = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) .setEntryId(position.getEntryId()) @@ -2579,29 +2846,33 @@ void switchToNewLedger(final LedgerHandle lh, final VoidCallback callback) { if (log.isDebugEnabled()) { log.debug("[{}] Switching cursor {} to ledger {}", ledger.getName(), name, lh.getId()); } + if (config.isEnableLruCacheMaxUnackedRanges()) { + callback.operationComplete(); + return; + } persistPositionMetaStore(lh.getId(), lastMarkDeleteEntry.newPosition, lastMarkDeleteEntry.properties, new MetaStoreCallback() { - @Override - public void operationComplete(Void result, Stat stat) { - log.info("[{}] Updated cursor {} with ledger id {} md-position={} rd-position={}", ledger.getName(), - name, lh.getId(), markDeletePosition, readPosition); - final LedgerHandle oldLedger = cursorLedger; - cursorLedger = lh; - isCursorLedgerReadOnly = false; - cursorLedgerStat = stat; + @Override + public void operationComplete(Void result, Stat stat) { + log.info("[{}] Updated cursor {} with ledger id {} md-position={} rd-position={}", ledger.getName(), + name, lh.getId(), markDeletePosition, readPosition); + final LedgerHandle oldLedger = cursorLedger; + cursorLedger = lh; + isCursorLedgerReadOnly = false; + cursorLedgerStat = stat; - // At this point the position had already been safely markdeleted - callback.operationComplete(); + // At this point the position had already been safely markdeleted + callback.operationComplete(); - asyncDeleteLedger(oldLedger); - } + asyncDeleteLedger(oldLedger); + } - @Override - public void operationFailed(MetaStoreException e) { - log.warn("[{}] Failed to update consumer {}", ledger.getName(), name, e); - callback.operationFailed(e); - } - }, false); + @Override + public void operationFailed(MetaStoreException e) { + log.warn("[{}] Failed to update consumer {}", ledger.getName(), name, e); + callback.operationFailed(e); + } + }, false); } /** @@ -2656,7 +2927,7 @@ void decrementPendingMarkDeleteCount() { if (state == State.SwitchingLedger) { // A metadata ledger switch was pending and now we can do it since we don't have any more // outstanding mark-delete requests - createNewMetadataLedger(); + createNewMetadataLedgerAndSwitch(); } } } diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index a5be8e4a4e379..0271535017fb4 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -82,6 +82,8 @@ message PositionInfo { // Store which index in the batch message has been deleted repeated BatchedEntryDeletionIndexInfo batchedEntryDeletionIndexInfo = 5; + + repeated MarkerIndexInfo markerIndexInfo = 6; } message NestedPositionInfo { @@ -99,6 +101,13 @@ message BatchedEntryDeletionIndexInfo { repeated int64 deleteSet = 2; } +message MarkerIndexInfo { + // the ledger we are acknowledging + required int64 targetLedgerId = 1; + // the position that saves the ack information + repeated NestedPositionInfo entryPosition= 2; +} + // Generic string and long tuple message LongProperty { required string name = 1; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index f706f3ac2562c..883b7d6ed58fc 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -2732,7 +2732,7 @@ public void testOutOfOrderDeletePersistenceAfterCrash() throws Exception { /** *
-     * Verifies that {@link ManagedCursorImpl#createNewMetadataLedger()} cleans up orphan ledgers if fails to switch new
+     * Verifies that {@link ManagedCursorImpl#createNewMetadataLedgerAndSwitch()} cleans up orphan ledgers if fails to switch new
      * ledger
      * 
* @throws Exception @@ -2745,7 +2745,7 @@ public void testLeakFailedLedgerOfManageCursor() throws Exception { ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); CountDownLatch latch = new CountDownLatch(1); - c1.createNewMetadataLedger(new VoidCallback() { + c1.createNewMetadataLedgerAndSwitch(new VoidCallback() { @Override public void operationComplete() { latch.countDown(); @@ -2771,7 +2771,7 @@ public void operationFailed(ManagedLedgerException exception) { CountDownLatch latch2 = new CountDownLatch(1); // create ledger will create ledgerId = 6 long ledgerId = 6; - c1.createNewMetadataLedger(new VoidCallback() { + c1.createNewMetadataLedgerAndSwitch(new VoidCallback() { @Override public void operationComplete() { latch2.countDown(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 9bca66351f3d0..31aa9a20b9b03 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -52,7 +52,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.RejectedExecutionException; @@ -2745,7 +2744,7 @@ private boolean updateCusorMetadataByCreatingMetadataLedger(MutableObject failed = new MutableObject<>(); failed.setValue(false); CountDownLatch createLedgerDoneLatch = new CountDownLatch(1); - cursor2.getValue().createNewMetadataLedger(new VoidCallback() { + cursor2.getValue().createNewMetadataLedgerAndSwitch(new VoidCallback() { @Override public void operationComplete() { 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 8615e58e3c205..5c35a9663a6b5 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 @@ -1517,6 +1517,17 @@ public class ServiceConfiguration implements PulsarConfiguration { doc = "Use Open Range-Set to cache unacked messages (it is memory efficient but it can take more cpu)" ) private boolean managedLedgerUnackedRangesOpenCacheSetEnabled = true; + @FieldContext( + category = CATEGORY_STORAGE_OFFLOADING, + doc = "If enabled, the maximum \"acknowledgment holes\" will not be limited. \n" + + "Only part of hotspot data will keep in memory by using LRU." + ) + private boolean enableLruCacheMaxUnackedRanges = true; + @FieldContext( + category = CATEGORY_STORAGE_OFFLOADING, + doc = "Max Bytes of UnackedRanges can keep in memory, only use for lru cache" + ) + private long maxLruUnackedRangesInMemoryBytes = 3145728; @FieldContext( dynamic = true, category = CATEGORY_STORAGE_ML, 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 7429e7b7da5a9..36e41ee94e40b 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 @@ -1349,6 +1349,8 @@ public CompletableFuture getManagedLedgerConfig(TopicName t managedLedgerConfig.setMaxUnackedRangesToPersist(serviceConfig.getManagedLedgerMaxUnackedRangesToPersist()); managedLedgerConfig.setMaxUnackedRangesToPersistInZk( serviceConfig.getManagedLedgerMaxUnackedRangesToPersistInZooKeeper()); + managedLedgerConfig.setEnableLruCacheMaxUnackedRanges(serviceConfig.isEnableLruCacheMaxUnackedRanges()); + managedLedgerConfig.setMaxUnackedRangesInMemoryBytes(serviceConfig.getMaxLruUnackedRangesInMemoryBytes()); managedLedgerConfig.setMaxEntriesPerLedger(serviceConfig.getManagedLedgerMaxEntriesPerLedger()); managedLedgerConfig.setMinimumRolloverTime(serviceConfig.getManagedLedgerMinLedgerRolloverTimeMinutes(), TimeUnit.MINUTES); From 909deb9df8944cd923c177fa9b8467b4801a1701 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Wed, 19 May 2021 11:20:37 +0800 Subject: [PATCH 02/18] Add recovery --- .../mledger/impl/ManagedCursorImpl.java | 75 ++++++++++++++----- 1 file changed, 56 insertions(+), 19 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 5fc9417bb9746..58b8738e9b6f4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -344,6 +344,35 @@ public void operationFailed(MetaStoreException e) { } protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallback callback) { + if (config.isEnableLruCacheMaxUnackedRanges()) { + recoverFromMultiEntry(info, callback); + } else { + recoverFromSingleEntry(info, callback); + } + } + + private void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback) { + ledger.mbean.startCursorLedgerOpenOp(); + long ledgerId = info.getCursorsLedgerId(); + OpenCallback openCallback = (rc, lh, ctx) -> { + log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc); + boolean shouldInitFromOldestEntry = shouldInitFromOldestEntry(info, callback, ledgerId, rc, lh); + if (shouldInitFromOldestEntry) { + return; + } + + + }; + try { + bookkeeper.asyncOpenLedger(ledgerId, digestType, config.getPassword(), openCallback, null); + } catch (Throwable t) { + log.error("[{}] Encountered error on opening cursor ledger {} for cursor {}", + ledger.getName(), ledgerId, name, t); + openCallback.openComplete(BKException.Code.UnexpectedConditionException, null, null); + } + } + + private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callback) { // Read the acknowledged position from the metadata ledger, then create // a new ledger and write the position into it ledger.mbean.startCursorLedgerOpenOp(); @@ -352,30 +381,13 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (log.isInfoEnabled()) { log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc); } - if (isBkErrorNotRecoverable(rc)) { - log.error("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, - BKException.getMessage(rc)); - // Rewind to oldest entry available - initialize(getRollbackPosition(info), Collections.emptyMap(), callback); - return; - } else if (rc != BKException.Code.OK) { - log.warn("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, - BKException.getMessage(rc)); - callback.operationFailed(new ManagedLedgerException(BKException.getMessage(rc))); + boolean shouldInitFromOldestEntry = shouldInitFromOldestEntry(info, callback, ledgerId, rc, lh); + if (shouldInitFromOldestEntry) { return; } // Read the last entry in the ledger long lastEntryInLedger = lh.getLastAddConfirmed(); - - if (lastEntryInLedger < 0) { - log.warn("[{}] Error reading from metadata ledger {} for consumer {}: No entries in ledger", - ledger.getName(), ledgerId, name); - // Rewind to last cursor snapshot available - initialize(getRollbackPosition(info), Collections.emptyMap(), callback); - return; - } - lh.asyncReadEntries(lastEntryInLedger, lastEntryInLedger, (rc1, lh1, seq, ctx1) -> { if (log.isDebugEnabled()) { log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed()); @@ -434,6 +446,31 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } } + private boolean shouldInitFromOldestEntry(ManagedCursorInfo info, VoidCallback callback, long ledgerId, int rc, + LedgerHandle lh) { + if (isBkErrorNotRecoverable(rc)) { + log.error("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, + BKException.getMessage(rc)); + // Rewind to oldest entry available + initialize(getRollbackPosition(info), Collections.emptyMap(), callback); + return true; + } else if (rc != BKException.Code.OK) { + log.warn("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, + BKException.getMessage(rc)); + callback.operationFailed(new ManagedLedgerException(BKException.getMessage(rc))); + return true; + } + + if (lh.getLastAddConfirmed() < 0) { + log.warn("[{}] Error reading from metadata ledger {} for consumer {}: No entries in ledger", + ledger.getName(), ledgerId, name); + // Rewind to last cursor snapshot available + initialize(getRollbackPosition(info), Collections.emptyMap(), callback); + return true; + } + return false; + } + private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { lock.writeLock().lock(); try { From 9ac80de7daad0b28c13bb74e8e8078ac2cee1993 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Wed, 19 May 2021 13:26:29 +0800 Subject: [PATCH 03/18] Support Recover --- .../mledger/impl/ManagedCursorImpl.java | 86 +++++++++++++------ .../src/main/proto/MLDataFormats.proto | 2 +- 2 files changed, 63 insertions(+), 25 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 58b8738e9b6f4..9598d190776f2 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -313,7 +313,7 @@ public void operationComplete(ManagedCursorInfo info, Stat stat) { PositionImpl recoveredPosition = new PositionImpl(info.getMarkDeleteLedgerId(), info.getMarkDeleteEntryId()); if (info.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(info.getIndividualDeletedMessagesList()); + recoverIndividualDeletedMessages(info.getIndividualDeletedMessagesList(), true); } Map recoveredProperties = Collections.emptyMap(); @@ -356,12 +356,41 @@ private void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback long ledgerId = info.getCursorsLedgerId(); OpenCallback openCallback = (rc, lh, ctx) -> { log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc); - boolean shouldInitFromOldestEntry = shouldInitFromOldestEntry(info, callback, ledgerId, rc, lh); - if (shouldInitFromOldestEntry) { + if (shouldRecoverFromOldestEntry(info, callback, ledgerId, rc, lh)) { return; } - - + Optional optional = getLastAvailableMarker(lh); + if (!optional.isPresent()) { + initialize(getRollbackPosition(info), Collections.emptyMap(), callback); + return; + } + PositionInfo markerPosition = optional.get(); + try { + for (MLDataFormats.MarkerIndexInfo markerIndexInfo : markerPosition.getMarkerIndexInfoList()) { + MLDataFormats.NestedPositionInfo nestedPositionInfo = markerIndexInfo.getEntryPosition(); + Enumeration entryEnumeration = lh.readEntries(nestedPositionInfo.getEntryId(), + nestedPositionInfo.getEntryId()); + if (entryEnumeration.hasMoreElements()) { + LedgerEntry entry = entryEnumeration.nextElement(); + PositionInfo entryPosition = PositionInfo.parseDelimitedFrom(entry.getEntryInputStream()); + if (entryPosition.getIndividualDeletedMessagesCount() > 0) { + recoverIndividualDeletedMessages(entryPosition.getIndividualDeletedMessagesList(), false); + } + if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null + && entryPosition.getBatchedEntryDeletionIndexInfoCount() > 0) { + recoverBatchDeletedIndexes(entryPosition.getBatchedEntryDeletionIndexInfoList(), false); + } + } + if (individualDeletedMessages.size() > config.getMaxUnackedRangesInMemoryBytes()) { + break; + } + } + } catch (Exception e) { + initialize(getRollbackPosition(info), Collections.emptyMap(), callback); + } + recoveredCursor(new PositionImpl(markerPosition.getLedgerId(), markerPosition.getEntryId()), + getRecoveredProperties(markerPosition), lh); + callback.operationComplete(); }; try { bookkeeper.asyncOpenLedger(ledgerId, digestType, config.getPassword(), openCallback, null); @@ -381,8 +410,8 @@ private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callbac if (log.isInfoEnabled()) { log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc); } - boolean shouldInitFromOldestEntry = shouldInitFromOldestEntry(info, callback, ledgerId, rc, lh); - if (shouldInitFromOldestEntry) { + boolean shouldFromOldestEntry = shouldRecoverFromOldestEntry(info, callback, ledgerId, rc, lh); + if (shouldFromOldestEntry) { return; } @@ -415,23 +444,15 @@ private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callbac return; } - Map recoveredProperties = Collections.emptyMap(); - if (positionInfo.getPropertiesCount() > 0) { - // Recover properties map - recoveredProperties = Maps.newHashMap(); - for (int i = 0; i < positionInfo.getPropertiesCount(); i++) { - LongProperty property = positionInfo.getProperties(i); - recoveredProperties.put(property.getName(), property.getValue()); - } - } + Map recoveredProperties = getRecoveredProperties(positionInfo); PositionImpl position = new PositionImpl(positionInfo); if (positionInfo.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); + recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList(), true); } if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { - recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); + recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList(), true); } recoveredCursor(position, recoveredProperties, lh); callback.operationComplete(); @@ -446,8 +467,21 @@ private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callbac } } - private boolean shouldInitFromOldestEntry(ManagedCursorInfo info, VoidCallback callback, long ledgerId, int rc, - LedgerHandle lh) { + private Map getRecoveredProperties(PositionInfo positionInfo) { + Map recoveredProperties = Collections.emptyMap(); + if (positionInfo.getPropertiesCount() > 0) { + // Recover properties map + recoveredProperties = Maps.newHashMap(); + for (int i = 0; i < positionInfo.getPropertiesCount(); i++) { + LongProperty property = positionInfo.getProperties(i); + recoveredProperties.put(property.getName(), property.getValue()); + } + } + return recoveredProperties; + } + + private boolean shouldRecoverFromOldestEntry(ManagedCursorInfo info, VoidCallback callback, long ledgerId, int rc, + LedgerHandle lh) { if (isBkErrorNotRecoverable(rc)) { log.error("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); @@ -471,10 +505,13 @@ private boolean shouldInitFromOldestEntry(ManagedCursorInfo info, VoidCallback c return false; } - private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { + private void recoverIndividualDeletedMessages(List individualDeletedMessagesList, + boolean cleanOldData) { lock.writeLock().lock(); try { - individualDeletedMessages.clear(); + if (cleanOldData) { + individualDeletedMessages.clear(); + } individualDeletedMessagesList.forEach(messageRange -> { MLDataFormats.NestedPositionInfo lowerEndpoint = messageRange.getLowerEndpoint(); MLDataFormats.NestedPositionInfo upperEndpoint = messageRange.getUpperEndpoint(); @@ -508,7 +545,8 @@ private void recoverIndividualDeletedMessages(List i } } - private void recoverBatchDeletedIndexes (List batchDeletedIndexInfoList) { + private void recoverBatchDeletedIndexes (List batchDeletedIndexInfoList + , boolean cleanOldData) { lock.writeLock().lock(); try { this.batchDeletedIndexes.clear(); @@ -2519,7 +2557,7 @@ private List buildMarkerIndexMap( Map rangeMarker) { MLDataFormats.MarkerIndexInfo.Builder builder = MLDataFormats.MarkerIndexInfo.newBuilder(); return rangeMarker.entrySet().stream().map((entry) -> - builder.setTargetLedgerId(entry.getKey()).addEntryPosition(entry.getValue()).build()) + builder.setTargetLedgerId(entry.getKey()).setEntryPosition(entry.getValue()).build()) .collect(Collectors.toList()); } diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index 0271535017fb4..532d3967f4c94 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -105,7 +105,7 @@ message MarkerIndexInfo { // the ledger we are acknowledging required int64 targetLedgerId = 1; // the position that saves the ack information - repeated NestedPositionInfo entryPosition= 2; + required NestedPositionInfo entryPosition= 2; } // Generic string and long tuple From d9522a70046457d22aab041cb302c71fda0c94da Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Thu, 20 May 2021 21:54:49 +0800 Subject: [PATCH 04/18] add unit test --- .../mledger/impl/LongPairRangeSetWrapper.java | 146 ----- .../mledger/impl/ManagedCursorImpl.java | 12 +- .../mledger/impl/RangeSetWrapper.java | 251 +++++++++ .../mledger/impl/RangeSetWrapperTest.java | 506 ++++++++++++++++++ .../ConcurrentOpenLongPairRangeSet.java | 2 +- .../util/collections/LongPairRangeSet.java | 8 + 6 files changed, 772 insertions(+), 153 deletions(-) delete mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LongPairRangeSetWrapper.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java create mode 100644 managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LongPairRangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LongPairRangeSetWrapper.java deleted file mode 100644 index ef2953f210488..0000000000000 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LongPairRangeSetWrapper.java +++ /dev/null @@ -1,146 +0,0 @@ -/** - * 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 com.google.common.collect.Range; -import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import java.util.concurrent.atomic.AtomicInteger; - -public class LongPairRangeSetWrapper> implements LongPairRangeSet { - - private final TreeMap> ledgerIdToPosition = new TreeMap<>(); - final LongPairConsumer positionRangeConverter; - final ManagedLedgerConfig config; - - public LongPairRangeSetWrapper(ManagedLedgerConfig config, LongPairConsumer positionRangeConverter) { - this.positionRangeConverter = positionRangeConverter; - this.config = config; - } - - @Override - public void addOpenClosed(long lowerKey, long lowerValue, long upperKey, long upperValue) { - LongPairRangeSet pairRangeSet = ledgerIdToPosition - .computeIfAbsent(upperKey, (key) -> getLongPairRangeSetImpl()); - if (config.isEnableLruCacheMaxUnackedRanges()) { - // lru switch - } else { - pairRangeSet.addOpenClosed(lowerKey, lowerValue, upperKey, upperValue); - } - } - - private LongPairRangeSet getLongPairRangeSetImpl() { - return config.isUnackedRangesOpenCacheSetEnabled() - ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter) - : new LongPairRangeSet.DefaultRangeSet<>(positionRangeConverter); - } - - @Override - public boolean contains(long key, long value) { - LongPairRangeSet longPairRangeSet = ledgerIdToPosition.get(key); - if (longPairRangeSet == null) { - return false; - } - return longPairRangeSet.contains(key, value); - } - - @Override - public Range rangeContaining(long key, long value) { - LongPairRangeSet longPairRangeSet = ledgerIdToPosition.get(key); - if (longPairRangeSet == null) { - return null; - } - return longPairRangeSet.rangeContaining(key, value); - } - - @Override - public void removeAtMost(long key, long value) { - LongPairRangeSet longPairRangeSet = ledgerIdToPosition.get(key); - if (longPairRangeSet == null) { - return; - } - longPairRangeSet.removeAtMost(key, value); - } - - @Override - public boolean isEmpty() { - return ledgerIdToPosition.isEmpty(); - } - - @Override - public void clear() { - ledgerIdToPosition.clear(); - } - - @Override - public Range span() { - Range rangeFirst = ledgerIdToPosition.firstEntry().getValue().span(); - Range rangeLast = ledgerIdToPosition.lastEntry().getValue().span(); - return Range.openClosed(rangeFirst.lowerEndpoint(), rangeLast.upperEndpoint()); - } - - @Override - public List> asRanges() { - List> ranges = new ArrayList<>(); - ledgerIdToPosition.values().forEach(ledgerIdToPosition -> ranges.addAll(ledgerIdToPosition.asRanges())); - return ranges; - } - - @Override - public void forEach(RangeProcessor action) { - ledgerIdToPosition.values().forEach(ledgerIdToPosition -> ledgerIdToPosition.forEach(action)); - } - - @Override - public void forEach(RangeProcessor action, LongPairConsumer consumer) { - ledgerIdToPosition.values().forEach(ledgerIdToPosition -> ledgerIdToPosition.forEach(action, consumer)); - } - - @Override - public int size() { - AtomicInteger size = new AtomicInteger(0); - ledgerIdToPosition.values().forEach(ledgerIdToPosition -> size.addAndGet(ledgerIdToPosition.size())); - return size.get(); - } - - @Override - public Range firstRange() { - Map.Entry> entry = ledgerIdToPosition.firstEntry(); - if (entry == null) { - return null; - } - return entry.getValue().firstRange(); - } - - @Override - public Range lastRange() { - Map.Entry> entry = ledgerIdToPosition.lastEntry(); - if (entry == null) { - return null; - } - return entry.getValue().lastRange(); - } -} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 9598d190776f2..070fd5e025e56 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -356,7 +356,7 @@ private void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback long ledgerId = info.getCursorsLedgerId(); OpenCallback openCallback = (rc, lh, ctx) -> { log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc); - if (shouldRecoverFromOldestEntry(info, callback, ledgerId, rc, lh)) { + if (shouldRecoverFromEldestEntry(info, callback, ledgerId, rc, lh)) { return; } Optional optional = getLastAvailableMarker(lh); @@ -410,8 +410,8 @@ private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callbac if (log.isInfoEnabled()) { log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc); } - boolean shouldFromOldestEntry = shouldRecoverFromOldestEntry(info, callback, ledgerId, rc, lh); - if (shouldFromOldestEntry) { + boolean shouldRecoverFromEldestEntry = shouldRecoverFromEldestEntry(info, callback, ledgerId, rc, lh); + if (shouldRecoverFromEldestEntry) { return; } @@ -424,7 +424,7 @@ private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callbac if (isBkErrorNotRecoverable(rc1)) { log.error("[{}] Error reading from metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc1)); - // Rewind to oldest entry available + // Rewind to eldest entry available initialize(getRollbackPosition(info), Collections.emptyMap(), callback); return; } else if (rc1 != BKException.Code.OK) { @@ -480,12 +480,12 @@ private Map getRecoveredProperties(PositionInfo positionInfo) { return recoveredProperties; } - private boolean shouldRecoverFromOldestEntry(ManagedCursorInfo info, VoidCallback callback, long ledgerId, int rc, + private boolean shouldRecoverFromEldestEntry(ManagedCursorInfo info, VoidCallback callback, long ledgerId, int rc, LedgerHandle lh) { if (isBkErrorNotRecoverable(rc)) { log.error("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); - // Rewind to oldest entry available + // Rewind to eldest entry available initialize(getRollbackPosition(info), Collections.emptyMap(), callback); return true; } else if (rc != BKException.Code.OK) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java new file mode 100644 index 0000000000000..00f97601772ed --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -0,0 +1,251 @@ +/** + * 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 com.google.common.collect.Range; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; +import org.apache.pulsar.common.util.collections.LongPairRangeSet; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +public class RangeSetWrapper> implements LongPairRangeSet, AutoCloseable { + + private final LongPairRangeSet rangeSet; + final LongPairConsumer rangeConverter; + final ManagedLedgerImpl managedLedger; + final ManagedLedgerConfig config; + final ScheduledFuture future; + final ConcurrentLinkedQueue pendingTouch = new ConcurrentLinkedQueue<>(); + final LruCache lruCounter = new LruCache<>(10, 0.75f, true); + + public RangeSetWrapper(ManagedLedgerImpl managedLedger, LongPairConsumer rangeConverter) { + this.rangeConverter = rangeConverter; + this.config = managedLedger.getConfig(); + this.managedLedger = managedLedger; + this.rangeSet = getLongPairRangeSetImpl(); + future = config.isEnableLruCacheMaxUnackedRanges() ? + managedLedger.getScheduledExecutor() + .scheduleAtFixedRate(new LruTask(this), 1, 1, TimeUnit.SECONDS) : null; + } + + @Override + public void addOpenClosed(long lowerKey, long lowerValue, long upperKey, long upperValue) { + lruTouch(upperKey); + rangeSet.addOpenClosed(lowerKey, lowerValue, upperKey, upperValue); + } + + private void lruTouch(long ledgerId) { + if (config.isEnableLruCacheMaxUnackedRanges()) { + pendingTouch.offer(ledgerId); + } + } + + private boolean isReachSwitchingThreshold() { + // every position has 3 long properties + return (long) size() * 24 > config.getMaxUnackedRangesInMemoryBytes(); + } + + private LongPairRangeSet getLongPairRangeSetImpl() { + return config.isUnackedRangesOpenCacheSetEnabled() + ? new ConcurrentOpenLongPairRangeSet<>(4096, rangeConverter) + : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter); + } + + @Override + public boolean contains(long key, long value) { + lruTouch(key); + return rangeSet.contains(key, value); + } + + @Override + public Range rangeContaining(long key, long value) { + lruTouch(key); + return rangeSet.rangeContaining(key, value); + } + + @Override + public void removeAtMost(long key, long value) { + lruTouch(key); + rangeSet.removeAtMost(key, value); + } + + public void add(Range range) { + if (!(rangeSet instanceof ConcurrentOpenLongPairRangeSet)) { + throw new UnsupportedOperationException("Only ConcurrentOpenLongPairRangeSet support this method"); + } + ConcurrentOpenLongPairRangeSet set = (ConcurrentOpenLongPairRangeSet) rangeSet; + set.add(range); + } + + public void remove(Range range) { + if (rangeSet instanceof ConcurrentOpenLongPairRangeSet) { + ConcurrentOpenLongPairRangeSet set = (ConcurrentOpenLongPairRangeSet) rangeSet; + Range longPairRange = (Range) range; + set.remove(longPairRange); + } else { + LongPairRangeSet.DefaultRangeSet set = (LongPairRangeSet.DefaultRangeSet) rangeSet; + set.remove(range); + } + } + + @Override + public boolean isEmpty() { + return rangeSet.isEmpty(); + } + + @Override + public void clear() { + rangeSet.clear(); + } + + @Override + public Range span() { + return rangeSet.span(); + } + + @Override + public List> asRanges() { + Collection> collection = rangeSet.asRanges(); + if (collection instanceof List) { + return (List>) collection; + } + return new ArrayList<>(collection); + } + + @Override + public void forEach(RangeProcessor action) { + rangeSet.forEach(action); + } + + @Override + public void forEach(RangeProcessor action, LongPairConsumer consumer) { + rangeSet.forEach(action, consumer); + } + + @Override + public int size() { + return rangeSet.size(); + } + + @Override + public Range firstRange() { + return rangeSet.firstRange(); + } + + @Override + public Range lastRange() { + return rangeSet.lastRange(); + } + + @Override + public void close() throws Exception { + if (future != null) { + future.cancel(true); + } + } + + @Override + public String toString() { + return rangeSet.toString(); + } + + private List> getPositionRangeInLedger(long ledgerId) { + LongPairRangeSet set = (LongPairRangeSet) rangeSet; + List> ranges = new ArrayList<>(); + set.forEach(range -> { + if (range.upperEndpoint().getLedgerId() == ledgerId) { + ranges.add(range); + } + return true; + }); + return ranges; + } + + class LruTask implements Runnable { + final RangeSetWrapper wrapper; + + public LruTask(RangeSetWrapper rangeSetWrapper) { + this.wrapper = rangeSetWrapper; + } + @Override + public void run() { + Long key = pendingTouch.poll(); + while (key != null) { + lruCounter.put(key, null); + key = pendingTouch.poll(); + } + + if (isReachSwitchingThreshold()) { + long eldestKey = lruCounter.removeEldestEntryAndGet(); + if (rangeSet instanceof ConcurrentOpenLongPairRangeSet) { + ConcurrentOpenLongPairRangeSet set = + (ConcurrentOpenLongPairRangeSet) rangeSet; + Range range = Range.openClosed(new LongPair(0, 0), new LongPair(0, 0)); + for (Range positionRange : getPositionRangeInLedger(eldestKey)) { + range.lowerEndpoint().setKey(positionRange.lowerEndpoint().ledgerId); + range.lowerEndpoint().setValue(positionRange.lowerEndpoint().entryId); + range.upperEndpoint().setKey(positionRange.upperEndpoint().ledgerId); + range.upperEndpoint().setValue(positionRange.upperEndpoint().entryId); + set.remove(range); + } + } else { + LongPairRangeSet.DefaultRangeSet set = + (LongPairRangeSet.DefaultRangeSet) rangeSet; + for (Range positionRange : getPositionRangeInLedger(eldestKey)) { + set.remove(positionRange); + } + } + } + } + } + + static class LruCache extends LinkedHashMap { + Map.Entry eldestEntry; + + public LruCache(int initialCapacity, + float loadFactor, + boolean accessOrder) { + super(initialCapacity, loadFactor, accessOrder); + } + + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + eldestEntry = eldest; + return false; + } + + public K removeEldestEntryAndGet() { + K key = eldestEntry.getKey(); + super.remove(key); + return key; + } + + public Map.Entry getEldestEntry() { + return eldestEntry; + } + } +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java new file mode 100644 index 0000000000000..53318ebc860d8 --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java @@ -0,0 +1,506 @@ +/** + * 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 com.google.common.collect.BoundType; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import com.google.common.collect.TreeRangeSet; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPair; +import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import java.util.List; +import java.util.Set; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; + +public class RangeSetWrapperTest { + + static final LongPairConsumer consumer = (key, value) -> new LongPair(key, value); + ManagedLedgerImpl managedLedger; + OrderedScheduler orderedScheduler; + RangeSetWrapper set; + ManagedLedgerConfig managedLedgerConfig; + + @BeforeMethod + public void setUp() { + managedLedger = mock(ManagedLedgerImpl.class); + initManagedLedgerConfig(); + orderedScheduler = OrderedScheduler.newSchedulerBuilder() + .numThreads(2).build(); + doReturn(managedLedgerConfig).when(managedLedger).getConfig(); + doReturn(orderedScheduler).when(managedLedger).getScheduledExecutor(); + } + + private void initManagedLedgerConfig() { + managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setEnableLruCacheMaxUnackedRanges(true); + managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled(true); + } + + @AfterMethod + public void clean() throws Exception { + if (orderedScheduler != null) { + orderedScheduler.shutdownNow(); + } + if (set != null) { + set.close(); + } + } + + + @Test + public void testAddForSameKey() { + doTestAddForSameKey(); + managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled(false); + doTestAddForSameKey(); + } + + private void doTestAddForSameKey() { + set = new RangeSetWrapper<>(managedLedger, consumer); + // add 0 to 5 + set.addOpenClosed(0, 0, 0, 5); + // add 8,9,10 + set.addOpenClosed(0, 8, 0, 8); + set.addOpenClosed(0, 9, 0, 9); + set.addOpenClosed(0, 10, 0, 10); + // add 98 to 99 and 102,105 + set.addOpenClosed(0, 98, 0, 99); + set.addOpenClosed(0, 102, 0, 106); + + List> ranges = set.asRanges(); + int count = 0; + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(0, 0), new LongPair(0, 5)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(0, 98), new LongPair(0, 99)))); + assertEquals(ranges.get(count), (Range.openClosed(new LongPair(0, 102), new LongPair(0, 106)))); + } + + @Test + public void testAddForDifferentKey() { + set = new RangeSetWrapper<>(managedLedger, consumer); + // [98,100],[(1,5),(1,5)],[(1,10,1,15)],[(1,20),(1,20)],[(2,0),(2,10)] + set.addOpenClosed(0, 98, 0, 99); + set.addOpenClosed(0, 100, 1, 5); + set.addOpenClosed(1, 10, 1, 15); + set.addOpenClosed(1, 20, 2, 10); + + List> ranges = set.asRanges(); + int count = 0; + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(0, 98), new LongPair(0, 99)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, -1), new LongPair(1, 5)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, 10), new LongPair(1, 15)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(2, -1), new LongPair(2, 10)))); + } + + @Test + public void testAddForDifferentKey2() { + managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled(false); + set = new RangeSetWrapper<>(managedLedger, consumer); + // [98,100],[(1,5),(1,5)],[(1,10,1,15)],[(1,20),(1,20)],[(2,0),(2,10)] + set.addOpenClosed(0, 98, 0, 99); + set.addOpenClosed(0, 100, 1, 5); + set.addOpenClosed(1, 10, 1, 15); + set.addOpenClosed(1, 20, 2, 10); + + List> ranges = set.asRanges(); + int count = 0; + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(0, 98), new LongPair(0, 99)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(0, 100), new LongPair(1, 5)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, 10), new LongPair(1, 15)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, 20), new LongPair(2, 10)))); + } + + @Test + public void testAddCompareCompareWithGuava() { + set = new RangeSetWrapper<>(managedLedger, consumer); + com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); + + // add 10K values for key 0 + int totalInsert = 10_000; + // add single values + for (int i = 0; i < totalInsert; i++) { + if (i % 3 == 0 || i % 6 == 0 || i % 8 == 0) { + LongPair lower = new LongPair(0, i - 1); + LongPair upper = new LongPair(0, i); + // set.add(Range.openClosed(lower, upper)); + set.addOpenClosed(lower.getKey(), lower.getValue(), upper.getKey(), upper.getValue()); + gSet.add(Range.openClosed(lower, upper)); + } + } + // add batches + for (int i = totalInsert; i < (totalInsert * 2); i++) { + if (i % 5 == 0) { + LongPair lower = new LongPair(0, i - 3 - 1); + LongPair upper = new LongPair(0, i + 3); + // set.add(Range.openClosed(lower, upper)); + set.addOpenClosed(lower.getKey(), lower.getValue(), upper.getKey(), upper.getValue()); + gSet.add(Range.openClosed(lower, upper)); + } + } + List> ranges = set.asRanges(); + Set> gRanges = gSet.asRanges(); + + List> gRangeConnected = getConnectedRange(gRanges); + assertEquals(gRangeConnected.size(), ranges.size()); + int i = 0; + for (Range range : gRangeConnected) { + assertEquals(range, ranges.get(i)); + i++; + } + } + + @Test + public void testDeleteCompareWithGuava() throws Exception { + RangeSetWrapper set = new RangeSetWrapper<>(managedLedger, consumer); + com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); + + // add 10K values for key 0 + int totalInsert = 10_000; + // add single values + List> removedRanges = Lists.newArrayList(); + for (int i = 0; i < totalInsert; i++) { + if (i % 3 == 0 || i % 7 == 0 || i % 11 == 0) { + continue; + } + LongPair lower = new LongPair(0, i - 1); + LongPair upper = new LongPair(0, i); + Range range = Range.openClosed(lower, upper); + // set.add(range); + set.addOpenClosed(lower.getKey(), lower.getValue(), upper.getKey(), upper.getValue()); + gSet.add(range); + if (i % 4 == 0) { + removedRanges.add(range); + } + } + // add batches + for (int i = totalInsert; i < (totalInsert * 2); i++) { + LongPair lower = new LongPair(0, i - 3 - 1); + LongPair upper = new LongPair(0, i + 3); + Range range = Range.openClosed(lower, upper); + if (i % 5 != 0) { + // set.add(range); + set.addOpenClosed(lower.getKey(), lower.getValue(), upper.getKey(), upper.getValue()); + gSet.add(range); + } + if (i % 4 == 0) { + removedRanges.add(range); + } + } + // remove records + for (Range range : removedRanges) { + set.remove(range); + gSet.remove(range); + } + + List> ranges = set.asRanges(); + Set> gRanges = gSet.asRanges(); + List> gRangeConnected = getConnectedRange(gRanges); + assertEquals(gRangeConnected.size(), ranges.size()); + int i = 0; + for (Range range : gRangeConnected) { + assertEquals(range, ranges.get(i)); + i++; + } + set.close(); + } + + @Test + public void testSpanWithGuava() { + set = new RangeSetWrapper<>(managedLedger, consumer); + com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); + set.addOpenClosed(0, 97, 0, 99); + gSet.add(Range.openClosed(new LongPair(0, 97), new LongPair(0, 99))); + set.addOpenClosed(0, 99, 1, 5); + gSet.add(Range.openClosed(new LongPair(0, 99), new LongPair(1, 5))); + assertEquals(set.span(), gSet.span()); + assertEquals(set.span(), Range.openClosed(new LongPair(0, 97), new LongPair(1, 5))); + + set.addOpenClosed(1, 9, 1, 15); + set.addOpenClosed(1, 19, 2, 10); + set.addOpenClosed(2, 24, 2, 28); + set.addOpenClosed(3, 11, 3, 20); + set.addOpenClosed(4, 11, 4, 20); + gSet.add(Range.openClosed(new LongPair(1, 9), new LongPair(1, 15))); + gSet.add(Range.openClosed(new LongPair(1, 19), new LongPair(2, 10))); + gSet.add(Range.openClosed(new LongPair(2, 24), new LongPair(2, 28))); + gSet.add(Range.openClosed(new LongPair(3, 11), new LongPair(3, 20))); + gSet.add(Range.openClosed(new LongPair(4, 11), new LongPair(4, 20))); + assertEquals(set.span(), gSet.span()); + assertEquals(set.span(), Range.openClosed(new LongPair(0, 97), new LongPair(4, 20))); + } + + @Test + public void testFirstRange() { + set = new RangeSetWrapper<>(managedLedger, consumer); + assertNull(set.firstRange()); + set.addOpenClosed(0, 97, 0, 99); + assertEquals(set.firstRange(), Range.openClosed(new LongPair(0, 97), new LongPair(0, 99))); + assertEquals(set.size(), 1); + set.addOpenClosed(0, 98, 0, 105); + assertEquals(set.firstRange(), Range.openClosed(new LongPair(0, 97), new LongPair(0, 105))); + assertEquals(set.size(), 1); + set.addOpenClosed(0, 5, 0, 75); + assertEquals(set.firstRange(), Range.openClosed(new LongPair(0, 5), new LongPair(0, 75))); + assertEquals(set.size(), 2); + } + + @Test + public void testLastRange() { + set = new RangeSetWrapper<>(managedLedger, consumer); + assertNull(set.lastRange()); + Range range = Range.openClosed(new LongPair(0, 97), new LongPair(0, 99)); + set.addOpenClosed(0, 97, 0, 99); + assertEquals(set.lastRange(), range); + assertEquals(set.size(), 1); + set.addOpenClosed(0, 98, 0, 105); + assertEquals(set.lastRange(), Range.openClosed(new LongPair(0, 97), new LongPair(0, 105))); + assertEquals(set.size(), 1); + range = Range.openClosed(new LongPair(1, 5), new LongPair(1, 75)); + set.addOpenClosed(1, 5, 1, 75); + assertEquals(set.lastRange(), range); + assertEquals(set.size(), 2); + range = Range.openClosed(new LongPair(1, 80), new LongPair(1, 120)); + set.addOpenClosed(1, 80, 1, 120); + assertEquals(set.lastRange(), range); + assertEquals(set.size(), 3); + } + + @Test + public void testToString() { + set = new RangeSetWrapper<>(managedLedger, consumer); + set.addOpenClosed(0, 97, 0, 99); + assertEquals(set.toString(), "[(0:97..0:99]]"); + set.addOpenClosed(0, 98, 0, 105); + assertEquals(set.toString(), "[(0:97..0:105]]"); + set.addOpenClosed(0, 5, 0, 75); + assertEquals(set.toString(), "[(0:5..0:75],(0:97..0:105]]"); + } + + @Test + public void testDeleteForDifferentKey() { + set = new RangeSetWrapper<>(managedLedger, consumer); + set.addOpenClosed(0, 97, 0, 99); + set.addOpenClosed(0, 99, 1, 5); + set.addOpenClosed(1, 9, 1, 15); + set.addOpenClosed(1, 19, 2, 10); + set.addOpenClosed(2, 24, 2, 28); + set.addOpenClosed(3, 11, 3, 20); + set.addOpenClosed(4, 11, 4, 20); + + // delete only (0,100) + set.remove(Range.open(new LongPair(0, 99), new LongPair(0, 105))); + + /** + * delete all keys from [2,27]->[4,15] : remaining [2,25..26,28], [4,16..20] + */ + set.remove(Range.closed(new LongPair(2, 27), new LongPair(4, 15))); + + List> ranges = set.asRanges(); + int count = 0; + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(0, 97), new LongPair(0, 99)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, -1), new LongPair(1, 5)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, 9), new LongPair(1, 15)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(2, -1), new LongPair(2, 10)))); + + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(2, 24), new LongPair(2, 26)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(2, 27), new LongPair(2, 28)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(4, 15), new LongPair(4, 20)))); + } + + @Test + public void testDeleteWithAtMost() { + set = new RangeSetWrapper<>(managedLedger, consumer); + set.addOpenClosed(0, 98, 0, 99); + set.addOpenClosed(0, 100, 1, 5); + set.addOpenClosed(1, 10, 1, 15); + set.addOpenClosed(1, 20, 2, 10); + set.addOpenClosed(2, 25, 2, 28); + set.addOpenClosed(3, 12, 3, 20); + set.addOpenClosed(4, 12, 4, 20); + + // delete only (0,100) + set.remove(Range.open(new LongPair(0, 99), new LongPair(0, 105))); + + /** + * delete all keys from [2,27]->[4,15] : remaining [2,25..26,28], [4,16..20] + */ + set.remove(Range.atMost(new LongPair(2, 27))); + + List> ranges = set.asRanges(); + int count = 0; + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(2, 27), new LongPair(2, 28)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(3, 12), new LongPair(3, 20)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(4, 12), new LongPair(4, 20)))); + } + + @Test + public void testDeleteWithAtMost2() { + set = new RangeSetWrapper<>(managedLedger, consumer); + set.addOpenClosed(0, 98, 0, 99); + set.addOpenClosed(0, 100, 1, 5); + set.addOpenClosed(1, 10, 1, 15); + set.addOpenClosed(1, 20, 2, 10); + set.addOpenClosed(2, 25, 2, 28); + set.addOpenClosed(3, 12, 3, 20); + set.addOpenClosed(4, 12, 4, 20); + + // delete only (0,100) + set.remove(Range.closed(new LongPair(0, 0), new LongPair(0, Integer.MAX_VALUE - 1))); + + List> ranges = set.asRanges(); + int count = 0; + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, -1), new LongPair(1, 5)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, 10), new LongPair(1, 15)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(2, -1), new LongPair(2, 10)))); + assertEquals(ranges.get(count), (Range.openClosed(new LongPair(2, 25), new LongPair(2, 28)))); + + managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled(false); + set = new RangeSetWrapper<>(managedLedger, consumer); + set.addOpenClosed(0, 98, 0, 99); + set.addOpenClosed(0, 100, 1, 5); + set.addOpenClosed(1, 10, 1, 15); + set.addOpenClosed(1, 20, 2, 10); + set.addOpenClosed(2, 25, 2, 28); + set.addOpenClosed(3, 12, 3, 20); + set.addOpenClosed(4, 12, 4, 20); + + set.remove(Range.openClosed(new LongPair(0, 0), new LongPair(0, Integer.MAX_VALUE - 1))); + ranges = set.asRanges(); + count = 0; + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(0, Integer.MAX_VALUE - 1), new LongPair(1, 5)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, 10), new LongPair(1, 15)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, 20), new LongPair(2, 10)))); + assertEquals(ranges.get(count), (Range.openClosed(new LongPair(2, 25), new LongPair(2, 28)))); + } + + @Test + public void testDeleteWithLeastMost() { + set = new RangeSetWrapper<>(managedLedger, consumer); + set.addOpenClosed(0, 98, 0, 99); + set.addOpenClosed(0, 100, 1, 5); + set.addOpenClosed(1, 10, 1, 15); + set.addOpenClosed(1, 20, 2, 10); + set.addOpenClosed(2, 25, 2, 28); + set.addOpenClosed(2, 12, 3, 20); + set.addOpenClosed(4, 12, 4, 20); + + // delete only (0,100) + set.remove(Range.open(new LongPair(0, 99), new LongPair(0, 105))); + + /** + * delete all keys from [2,27]->[4,15] : remaining [2,25..26,28], [4,16..20] + */ + set.remove(Range.atLeast(new LongPair(2, 27))); + + List> ranges = set.asRanges(); + int count = 0; + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(0, 98), new LongPair(0, 99)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, -1), new LongPair(1, 5)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(1, 10), new LongPair(1, 15)))); + assertEquals(ranges.get(count++), (Range.openClosed(new LongPair(2, -1), new LongPair(2, 10)))); + assertEquals(ranges.get(count), (Range.openClosed(new LongPair(2, 12), new LongPair(2, 26)))); + } + + @Test + public void testRangeContaining() { + set = new RangeSetWrapper<>(managedLedger, consumer); + set.add(Range.closed(new LongPair(0, 98), new LongPair(0, 99))); + set.add(Range.closed(new LongPair(0, 100), new LongPair(1, 5))); + com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); + gSet.add(Range.closed(new LongPair(0, 98), new LongPair(0, 100))); + gSet.add(Range.closed(new LongPair(0, 101), new LongPair(1, 5))); + set.add(Range.closed(new LongPair(1, 10), new LongPair(1, 15))); + set.add(Range.closed(new LongPair(1, 20), new LongPair(2, 10))); + set.add(Range.closed(new LongPair(2, 25), new LongPair(2, 28))); + set.add(Range.closed(new LongPair(3, 12), new LongPair(3, 20))); + set.add(Range.closed(new LongPair(4, 12), new LongPair(4, 20))); + gSet.add(Range.closed(new LongPair(1, 10), new LongPair(1, 15))); + gSet.add(Range.closed(new LongPair(1, 20), new LongPair(2, 10))); + gSet.add(Range.closed(new LongPair(2, 25), new LongPair(2, 28))); + gSet.add(Range.closed(new LongPair(3, 12), new LongPair(3, 20))); + gSet.add(Range.closed(new LongPair(4, 12), new LongPair(4, 20))); + + LongPair position = new LongPair(0, 99); + assertEquals(set.rangeContaining(position.getKey(), position.getValue()), + Range.closed(new LongPair(0, 98), new LongPair(0, 100))); + assertEquals(set.rangeContaining(position.getKey(), position.getValue()), gSet.rangeContaining(position)); + + position = new LongPair(2, 30); + assertNull(set.rangeContaining(position.getKey(), position.getValue())); + assertEquals(set.rangeContaining(position.getKey(), position.getValue()), gSet.rangeContaining(position)); + + position = new LongPair(3, 13); + assertEquals(set.rangeContaining(position.getKey(), position.getValue()), + Range.closed(new LongPair(3, 12), new LongPair(3, 20))); + assertEquals(set.rangeContaining(position.getKey(), position.getValue()), gSet.rangeContaining(position)); + + position = new LongPair(3, 22); + assertNull(set.rangeContaining(position.getKey(), position.getValue())); + assertEquals(set.rangeContaining(position.getKey(), position.getValue()), gSet.rangeContaining(position)); + } + + @Test + public void testCacheFlagConflict() { + set = new RangeSetWrapper<>(managedLedger, consumer); + set.add(Range.openClosed(new LongPair(0, 1), new LongPair(0, 2))); + set.add(Range.openClosed(new LongPair(0, 3), new LongPair(0, 4))); + assertEquals(set.toString(), "[(0:1..0:2],(0:3..0:4]]"); + assertEquals(set.size(), 2); + } + + private List> getConnectedRange(Set> gRanges) { + List> gRangeConnected = Lists.newArrayList(); + Range lastRange = null; + for (Range range : gRanges) { + if (lastRange == null) { + lastRange = range; + continue; + } + LongPair previousUpper = lastRange.upperEndpoint(); + LongPair currentLower = range.lowerEndpoint(); + int previousUpperValue = (int) (lastRange.upperBoundType().equals(BoundType.CLOSED) + ? previousUpper.getValue() + : previousUpper.getValue() - 1); + int currentLowerValue = (int) (range.lowerBoundType().equals(BoundType.CLOSED) ? currentLower.getValue() + : currentLower.getValue() + 1); + boolean connected = (previousUpper.getKey() == currentLower.getKey()) + ? (previousUpperValue >= currentLowerValue) + : false; + if (connected) { + lastRange = Range.closed(lastRange.lowerEndpoint(), range.upperEndpoint()); + } else { + gRangeConnected.add(lastRange); + lastRange = range; + } + } + int lowerOpenValue = (int) (lastRange.lowerBoundType().equals(BoundType.CLOSED) + ? (lastRange.lowerEndpoint().getValue() - 1) + : lastRange.lowerEndpoint().getValue()); + lastRange = Range.openClosed(new LongPair(lastRange.lowerEndpoint().getKey(), lowerOpenValue), + lastRange.upperEndpoint()); + gRangeConnected.add(lastRange); + return gRangeConnected; + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java index dddc938218245..34cb3280f9907 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java @@ -343,7 +343,7 @@ public void remove(Range range) { // remove all the keys between two endpoint keys rangeBitSetMap.forEach((key, set) -> { - if (lowerEndpoint.getKey() == upperEndpoint.getKey()) { + if (lowerEndpoint.getKey() == upperEndpoint.getKey() && key == upperEndpoint.getKey()) { set.clear((int) lower, (int) upper + 1); } else { // eg: remove-range: [(3,5) - (5,5)] -> Delete all items from 3,6->3,N,4.*,5,0->5,5 diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java index 92ae3a659201b..b79e3baa32245 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java @@ -174,6 +174,14 @@ public long getValue() { return this.value; } + public void setKey(long key) { + this.key = key; + } + + public void setValue(long value) { + this.value = value; + } + @Override public int compareTo(LongPair o) { return ComparisonChain.start().compare(key, o.getKey()).compare(value, o.getValue()).result(); From b3c3f9c95f43dbc26950f1b509431fb8beeeefa4 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Sat, 22 May 2021 18:06:55 +0800 Subject: [PATCH 05/18] add unit test --- .../mledger/impl/ManagedCursorImpl.java | 103 ++++++--- .../mledger/impl/RangeSetWrapper.java | 217 +++++++++++------- .../mledger/impl/MultiEntryPositionTest.java | 173 ++++++++++++++ .../mledger/impl/RangeSetWrapperTest.java | 83 +++++-- .../ConcurrentOpenLongPairRangeSet.java | 8 +- 5 files changed, 454 insertions(+), 130 deletions(-) create mode 100644 managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 070fd5e025e56..c69041995513f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -104,7 +104,6 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer; import org.apache.pulsar.metadata.api.Stat; @@ -203,7 +202,7 @@ public class ManagedCursorImpl implements ManagedCursor { private long entriesReadSize; private int individualDeletedMessagesSerializedSize; - class MarkDeleteEntry { + static class MarkDeleteEntry { final PositionImpl newPosition; final MarkDeleteCallback callback; final Object ctx; @@ -261,9 +260,7 @@ public interface VoidCallback { this.config = config; this.ledger = ledger; this.name = cursorName; - this.individualDeletedMessages = config.isUnackedRangesOpenCacheSetEnabled() - ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter) - : new LongPairRangeSet.DefaultRangeSet<>(positionRangeConverter); + this.individualDeletedMessages = new RangeSetWrapper<>(positionRangeConverter, this); if (config.isDeletionAtBatchIndexLevelEnabled()) { this.batchDeletedIndexes = new ConcurrentSkipListMap<>(); } else { @@ -505,7 +502,7 @@ private boolean shouldRecoverFromEldestEntry(ManagedCursorInfo info, VoidCallbac return false; } - private void recoverIndividualDeletedMessages(List individualDeletedMessagesList, + protected void recoverIndividualDeletedMessages(List individualDeletedMessagesList, boolean cleanOldData) { lock.writeLock().lock(); try { @@ -673,6 +670,16 @@ public void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadE ledger.asyncReadEntries(op); } + public void asyncReadEntry(PositionImpl readPosition, Object ctx, ReadEntriesCallback callback) { + if (isClosed()) { + callback.readEntriesFailed(new ManagedLedgerException("Cursor was already closed"), ctx); + return; + } + PENDING_READ_OPS_UPDATER.incrementAndGet(this); + OpReadEntry op = OpReadEntry.create(this, readPosition, 1, callback, ctx, readPosition); + ledger.asyncReadEntries(op); + } + @Override public Entry getNthEntry(int n, IndividualDeletedEntries deletedEntries) throws InterruptedException, ManagedLedgerException { @@ -2189,6 +2196,14 @@ public void close() throws InterruptedException, ManagedLedgerException { class Result { ManagedLedgerException exception = null; } + if (individualDeletedMessages instanceof RangeSetWrapper) { + RangeSetWrapper wrapper = (RangeSetWrapper) individualDeletedMessages; + try { + wrapper.close(); + } catch (Exception e) { + log.warn("Closing RangeSetWrapper failed", e); + } + } final Result result = new Result(); final CountDownLatch latch = new CountDownLatch(1); @@ -2450,7 +2465,7 @@ void createNewMetadataLedgerAndSwitch(final VoidCallback callback) { deleteLedger(newLedgerHandle); callback.operationFailed(createManagedLedgerException(e)); } else { - callback.operationComplete(); + trySwitchToNewLedger(newLedgerHandle, callback); } }); return; @@ -2464,20 +2479,7 @@ public void operationComplete() { log.debug("[{}] Persisted position {} for cursor {}", ledger.getName(), mdEntry.newPosition, name); } - switchToNewLedger(newLedgerHandle, new VoidCallback() { - @Override - public void operationComplete() { - callback.operationComplete(); - } - - @Override - public void operationFailed(ManagedLedgerException exception) { - // it means it failed to switch the newly created ledger so, it should be - // deleted to prevent leak - deleteLedger(newLedgerHandle); - callback.operationFailed(exception); - } - }); + trySwitchToNewLedger(newLedgerHandle, callback); } @Override @@ -2496,6 +2498,23 @@ public void operationFailed(ManagedLedgerException exception) { } + private void trySwitchToNewLedger(LedgerHandle newLedgerHandle, VoidCallback callback) { + switchToNewLedger(newLedgerHandle, new VoidCallback() { + @Override + public void operationComplete() { + callback.operationComplete(); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + // it means it failed to switch the newly created ledger so, it should be + // deleted to prevent leak + deleteLedger(newLedgerHandle); + callback.operationFailed(exception); + } + }); + } + private void deleteLedger(LedgerHandle ledgerHandle) { ledger.mbean.startCursorLedgerDeleteOp(); bookkeeper.asyncDeleteLedger(ledgerHandle.getId(), (int rc, Object ctx) -> { @@ -2507,7 +2526,7 @@ private void deleteLedger(LedgerHandle ledgerHandle) { }, null); } - private CompletableFuture doCreateNewMetadataLedger() { + CompletableFuture doCreateNewMetadataLedger() { CompletableFuture future = new CompletableFuture<>(); ledger.asyncCreateLedger(bookkeeper, config, digestType, (rc, lh, ctx) -> { @@ -2738,7 +2757,7 @@ private CompletableFuture checkIfNeedCreateNewLruLedgerAndSwitch(L private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle, LedgerHandle newHandle) { CompletableFuture result = new CompletableFuture<>(); - // copy entries to new ledger + // copy all entries in marker to new ledger List> futures = Collections.synchronizedList(new ArrayList<>()); Map dirtyLedgerRecorder = new ConcurrentHashMap<>(); try { @@ -2746,8 +2765,9 @@ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle MLDataFormats.NestedPositionInfo position = longPositionEntry.getValue(); Enumeration entryEnumeration = oldHandle.readEntries(position.getEntryId(), position.getEntryId()); - if (entryEnumeration == null) { - log.warn("entry is deleted {}:{}", oldHandle.getLedgerMetadata().getLedgerId(), position.getEntryId()); + if (entryEnumeration == null || !entryEnumeration.hasMoreElements()) { + log.warn("can not find entry {}:{}", oldHandle.getLedgerMetadata().getLedgerId(), + position.getEntryId()); continue; } final CompletableFuture future = new CompletableFuture<>(); @@ -2814,7 +2834,7 @@ private Optional getLastAvailableMarker(LedgerHandle ledgerHandle) return Optional.empty(); } - private Map> getDeletionIndexInfosGroupByLedgerId() { + Map> getDeletionIndexInfosGroupByLedgerId() { List batchDeletionIndexInfos = buildBatchEntryDeletionIndexInfoList(); Map> ledgerIdToIndexInfo = new HashMap<>(); @@ -2827,7 +2847,7 @@ private Map> getDeletion return ledgerIdToIndexInfo; } - private Map> getRangeGroupByLedgerId() { + Map> getRangeGroupByLedgerId() { List rangeList = buildIndividualDeletedMessageRanges(); Map> ledgerIdToMessageRange = new HashMap<>(); rangeList.forEach(messageRange -> { @@ -2921,10 +2941,6 @@ void switchToNewLedger(final LedgerHandle lh, final VoidCallback callback) { if (log.isDebugEnabled()) { log.debug("[{}] Switching cursor {} to ledger {}", ledger.getName(), name, lh.getId()); } - if (config.isEnableLruCacheMaxUnackedRanges()) { - callback.operationComplete(); - return; - } persistPositionMetaStore(lh.getId(), lastMarkDeleteEntry.newPosition, lastMarkDeleteEntry.properties, new MetaStoreCallback() { @Override @@ -3133,6 +3149,26 @@ public long getCursorLedger() { return lh != null ? lh.getId() : -1; } + @VisibleForTesting + LedgerHandle getCursorLedgerHandle() { + return cursorLedger; + } + + @VisibleForTesting + MarkDeleteEntry getLastMarkDeleteEntry() { + return lastMarkDeleteEntry; + } + + @VisibleForTesting + void setLastMarkDeleteEntry(MarkDeleteEntry lastMarkDeleteEntry) { + this.lastMarkDeleteEntry = lastMarkDeleteEntry; + } + + @VisibleForTesting + ConcurrentSkipListMap getBatchDeletedIndexes() { + return batchDeletedIndexes; + } + public long getCursorLedgerLastEntry() { LedgerHandle lh = cursorLedger; return lh != null ? lh.getLastAddConfirmed() : -1; @@ -3330,5 +3366,10 @@ public boolean checkAndUpdateReadPositionChanged() { return isReadPositionOnTail || isReadPositionChanged; } + @VisibleForTesting + protected Map getRangeMarker() { + return rangeMarker; + } + private static final Logger log = LoggerFactory.getLogger(ManagedCursorImpl.class); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index 00f97601772ed..c40d287ab0434 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -20,36 +20,55 @@ import com.google.common.collect.Range; +import org.apache.bookkeeper.client.LedgerEntry; +import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Collection; +import java.util.Enumeration; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.Set; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import static com.google.common.base.Preconditions.checkNotNull; + public class RangeSetWrapper> implements LongPairRangeSet, AutoCloseable { + private static final Logger log = LoggerFactory.getLogger(RangeSetWrapper.class); private final LongPairRangeSet rangeSet; - final LongPairConsumer rangeConverter; - final ManagedLedgerImpl managedLedger; - final ManagedLedgerConfig config; - final ScheduledFuture future; - final ConcurrentLinkedQueue pendingTouch = new ConcurrentLinkedQueue<>(); - final LruCache lruCounter = new LruCache<>(10, 0.75f, true); - - public RangeSetWrapper(ManagedLedgerImpl managedLedger, LongPairConsumer rangeConverter) { + private final ManagedCursorImpl managedCursor; + private final LongPairConsumer rangeConverter; + private final ManagedLedgerConfig config; + private final ScheduledFuture future; + private final OrderedScheduler scheduler; + private final LruCache lruCounter = new LruCache<>(); + + public RangeSetWrapper(LongPairConsumer rangeConverter, ManagedCursorImpl managedCursor) { + checkNotNull(managedCursor); + this.config = managedCursor.getManagedLedger().getConfig(); + this.managedCursor = managedCursor; this.rangeConverter = rangeConverter; - this.config = managedLedger.getConfig(); - this.managedLedger = managedLedger; - this.rangeSet = getLongPairRangeSetImpl(); - future = config.isEnableLruCacheMaxUnackedRanges() ? - managedLedger.getScheduledExecutor() - .scheduleAtFixedRate(new LruTask(this), 1, 1, TimeUnit.SECONDS) : null; + this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() + ? new ConcurrentOpenLongPairRangeSet<>(4096, rangeConverter) + : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter); + if (managedCursor.getManagedLedger() instanceof ManagedLedgerImpl && config.isEnableLruCacheMaxUnackedRanges()) { + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) managedCursor.getManagedLedger(); + scheduler = managedLedger.getScheduledExecutor(); + future = scheduler.scheduleWithFixedDelay( + new LruTask(), 1, 5, TimeUnit.SECONDS); + } else { + future = null; + scheduler = null; + } } @Override @@ -60,31 +79,47 @@ public void addOpenClosed(long lowerKey, long lowerValue, long upperKey, long up private void lruTouch(long ledgerId) { if (config.isEnableLruCacheMaxUnackedRanges()) { - pendingTouch.offer(ledgerId); + scheduler.execute(() -> lruCounter.touch(ledgerId)); } } - private boolean isReachSwitchingThreshold() { + public boolean isReachLruSwitchThreshold() { + if (rangeSet instanceof ConcurrentOpenLongPairRangeSet) { + ConcurrentOpenLongPairRangeSet set = (ConcurrentOpenLongPairRangeSet) rangeSet; + return set.getRangeBitSetMapMemorySize() > config.getMaxUnackedRangesInMemoryBytes(); + } // every position has 3 long properties return (long) size() * 24 > config.getMaxUnackedRangesInMemoryBytes(); } - private LongPairRangeSet getLongPairRangeSetImpl() { - return config.isUnackedRangesOpenCacheSetEnabled() - ? new ConcurrentOpenLongPairRangeSet<>(4096, rangeConverter) - : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter); + public LongPairRangeSet getRangeSet() { + return rangeSet; + } + + public LruCache getLruCounter() { + return lruCounter; } @Override public boolean contains(long key, long value) { lruTouch(key); - return rangeSet.contains(key, value); + boolean isContain = rangeSet.contains(key, value); + if (!isContain && managedCursor.getRangeMarker().containsKey(key)) { + tryLoadLruRangeFromLedger(key); + return rangeSet.contains(key, value); + } + return isContain; } @Override - public Range rangeContaining(long key, long value) { + public Range rangeContaining(long key, long value) { lruTouch(key); - return rangeSet.rangeContaining(key, value); + Range range = rangeSet.rangeContaining(key, value); + if(range == null && managedCursor.getRangeMarker().containsKey(key)){ + tryLoadLruRangeFromLedger(key); + return rangeSet.rangeContaining(key, value); + } + return range; } @Override @@ -93,6 +128,28 @@ public void removeAtMost(long key, long value) { rangeSet.removeAtMost(key, value); } + private void tryLoadLruRangeFromLedger(long key) { + if (config.isEnableLruCacheMaxUnackedRanges()) { + MLDataFormats.NestedPositionInfo positionInfo = managedCursor.getRangeMarker().get(key); + if (positionInfo == null) { + return; + } + try { + Enumeration entryEnumeration = managedCursor.getCursorLedgerHandle() + .readEntries(positionInfo.getLedgerId(), positionInfo.getEntryId()); + if(entryEnumeration.hasMoreElements()){ + LedgerEntry ledgerEntry = entryEnumeration.nextElement(); + MLDataFormats.PositionInfo entryPosition = + MLDataFormats.PositionInfo.parseDelimitedFrom(ledgerEntry.getEntryInputStream()); + managedCursor.recoverIndividualDeletedMessages(entryPosition.getIndividualDeletedMessagesList(), + false); + } + } catch (Exception e) { + log.error("load lru entry failed", e); + } + } + } + public void add(Range range) { if (!(rangeSet instanceof ConcurrentOpenLongPairRangeSet)) { throw new UnsupportedOperationException("Only ConcurrentOpenLongPairRangeSet support this method"); @@ -103,11 +160,11 @@ public void add(Range range) { public void remove(Range range) { if (rangeSet instanceof ConcurrentOpenLongPairRangeSet) { - ConcurrentOpenLongPairRangeSet set = (ConcurrentOpenLongPairRangeSet) rangeSet; + ConcurrentOpenLongPairRangeSet set = (ConcurrentOpenLongPairRangeSet) rangeSet; Range longPairRange = (Range) range; set.remove(longPairRange); } else { - LongPairRangeSet.DefaultRangeSet set = (LongPairRangeSet.DefaultRangeSet) rangeSet; + LongPairRangeSet.DefaultRangeSet set = (LongPairRangeSet.DefaultRangeSet) rangeSet; set.remove(range); } } @@ -163,7 +220,7 @@ public Range lastRange() { @Override public void close() throws Exception { - if (future != null) { + if (future != null && !future.isCancelled()) { future.cancel(true); } } @@ -173,79 +230,83 @@ public String toString() { return rangeSet.toString(); } - private List> getPositionRangeInLedger(long ledgerId) { - LongPairRangeSet set = (LongPairRangeSet) rangeSet; - List> ranges = new ArrayList<>(); - set.forEach(range -> { - if (range.upperEndpoint().getLedgerId() == ledgerId) { - ranges.add(range); - } - return true; - }); - return ranges; - } - class LruTask implements Runnable { - final RangeSetWrapper wrapper; - public LruTask(RangeSetWrapper rangeSetWrapper) { - this.wrapper = rangeSetWrapper; - } @Override public void run() { - Long key = pendingTouch.poll(); - while (key != null) { - lruCounter.put(key, null); - key = pendingTouch.poll(); + log.debug("start schedule LruTask"); + // remove invalid key which is removed in rangeSet + Range span = rangeSet.span(); + for (Long ledgerId : lruCounter.getKeys()) { + if (span.lowerEndpoint().compareTo(rangeConverter.apply(ledgerId, Integer.MAX_VALUE - 1)) > 0) { + lruCounter.remove(ledgerId); + log.info("LruTask remove invalid key {}", ledgerId); + } else { + break; + } } - - if (isReachSwitchingThreshold()) { + if (isReachLruSwitchThreshold() && lruCounter.size() > 1) { long eldestKey = lruCounter.removeEldestEntryAndGet(); + log.info("Reach switching threshold, try to remove eldest ledger {}, range size {}", + eldestKey, rangeSet.size()); if (rangeSet instanceof ConcurrentOpenLongPairRangeSet) { - ConcurrentOpenLongPairRangeSet set = - (ConcurrentOpenLongPairRangeSet) rangeSet; - Range range = Range.openClosed(new LongPair(0, 0), new LongPair(0, 0)); - for (Range positionRange : getPositionRangeInLedger(eldestKey)) { - range.lowerEndpoint().setKey(positionRange.lowerEndpoint().ledgerId); - range.lowerEndpoint().setValue(positionRange.lowerEndpoint().entryId); - range.upperEndpoint().setKey(positionRange.upperEndpoint().ledgerId); - range.upperEndpoint().setValue(positionRange.upperEndpoint().entryId); - set.remove(range); - } + ConcurrentOpenLongPairRangeSet set = + (ConcurrentOpenLongPairRangeSet) rangeSet; + set.remove(Range.openClosed(new LongPair(eldestKey, 0), new LongPair(eldestKey, + Integer.MAX_VALUE - 1))); } else { - LongPairRangeSet.DefaultRangeSet set = - (LongPairRangeSet.DefaultRangeSet) rangeSet; - for (Range positionRange : getPositionRangeInLedger(eldestKey)) { - set.remove(positionRange); - } + LongPairRangeSet.DefaultRangeSet set = + (LongPairRangeSet.DefaultRangeSet) rangeSet; + set.remove(Range.openClosed(rangeConverter.apply(eldestKey, 0), + rangeConverter.apply(eldestKey, Integer.MAX_VALUE - 1))); } } } } - static class LruCache extends LinkedHashMap { - Map.Entry eldestEntry; + static class LruCache { + private Field field; + final LinkedHashMap linkedHashMap; - public LruCache(int initialCapacity, - float loadFactor, - boolean accessOrder) { - super(initialCapacity, loadFactor, accessOrder); + public LruCache() { + linkedHashMap = new LinkedHashMap<>(10, 0.75f, true); } - @Override - protected boolean removeEldestEntry(Map.Entry eldest) { - eldestEntry = eldest; - return false; + public Set getKeys() { + return linkedHashMap.keySet(); + } + + public V remove(K key) { + return linkedHashMap.remove(key); + } + + public void touch(K key) { + linkedHashMap.putIfAbsent(key, null); } public K removeEldestEntryAndGet() { - K key = eldestEntry.getKey(); - super.remove(key); - return key; + Map.Entry entry = getEldestEntry(); + if (entry != null) { + linkedHashMap.remove(entry.getKey()); + return entry.getKey(); + } + return null; } public Map.Entry getEldestEntry() { - return eldestEntry; + try { + if (field == null) { + field = LinkedHashMap.class.getDeclaredField("head"); + field.setAccessible(true); + } + return (Map.Entry) field.get(linkedHashMap); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public int size() { + return linkedHashMap.size(); } } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java new file mode 100644 index 0000000000000..c7af0f96d0792 --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java @@ -0,0 +1,173 @@ +/** + * 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.client.LedgerEntry; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.apache.pulsar.common.util.collections.BitSetRecyclable; +import org.awaitility.Awaitility; +import org.testng.annotations.Test; +import java.util.Arrays; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import static org.testng.AssertJUnit.assertEquals; + +public class MultiEntryPositionTest extends MockedBookKeeperTestCase { + + @Test + public void testGetRangeGroupByLedgerId() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setEnableLruCacheMaxUnackedRanges(true); + ManagedLedger ledger = factory.open("my_test_ledger", config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(0, 0, 0, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(1, 0, 1, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(2, 0, 2, 1); + + Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(); + + assertEquals(rangeGroupByLedgerId.size(), 3); + AtomicLong count = new AtomicLong(0); + rangeGroupByLedgerId.forEach((key, value) -> { + assertEquals(key.longValue(), count.get()); + assertEquals(value.size(), 1); + assertEquals(value.get(0).getLowerEndpoint().getLedgerId(), count.get()); + assertEquals(value.get(0).getLowerEndpoint().getEntryId(), 0); + assertEquals(value.get(0).getUpperEndpoint().getLedgerId(), count.getAndIncrement()); + assertEquals(value.get(0).getUpperEndpoint().getEntryId(), 1); + }); + + c1.close(); + ledger.close(); + } + + @Test + public void testGetDeletionIndexInfosGroupByLedgerId() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setEnableLruCacheMaxUnackedRanges(true); + ManagedLedger ledger = factory.open("my_test_ledger", config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + + List list = Arrays.asList(BitSetRecyclable.create(), + BitSetRecyclable.create(), + BitSetRecyclable.create()); + c1.getBatchDeletedIndexes().put(new PositionImpl(0,0), list.get(0)); + c1.getBatchDeletedIndexes().put(new PositionImpl(1,0), list.get(1)); + c1.getBatchDeletedIndexes().put(new PositionImpl(2,0), list.get(2)); + + Map> map = c1.getDeletionIndexInfosGroupByLedgerId(); + + assertEquals(map.size(), 3); + AtomicLong count = new AtomicLong(0); + map.forEach((key, value) -> { + assertEquals(key.longValue(), count.get()); + assertEquals(value.size(), 1); + assertEquals(value.get(0).getPosition().getLedgerId(), count.getAndIncrement()); + assertEquals(value.get(0).getPosition().getEntryId(), 0); + }); + + c1.close(); + ledger.close(); + list.forEach(BitSetRecyclable::recycle); + } + + @Test + public void testCopyLruEntriesToNewLedger() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setEnableLruCacheMaxUnackedRanges(true); + ManagedLedger ledger = factory.open("my_test_ledger", config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + //init new ledger + c1.startCreatingNewMetadataLedger(); + Awaitility.await().untilAsserted(() -> assertEquals(c1.getState(),"Open")); + LedgerHandle ledgerHandle = c1.getCursorLedgerHandle(); + long ledgerId = ledgerHandle.getId(); + // init IndividuallyDeletedMessages + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(0, 0, 0, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(1, 0, 1, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(2, 0, 2, 1); + Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(); + MLDataFormats.PositionInfo.Builder builder = + MLDataFormats.PositionInfo.newBuilder().setLedgerId(1).setEntryId(0); + Map rangeMarker = new HashMap<>(); + for (int i = 0; i < 3; i++) { + if (builder.getIndividualDeletedMessagesList().size() > 0) { + builder.removeIndividualDeletedMessages(0); + } + builder.addIndividualDeletedMessages(rangeGroupByLedgerId.get((long) i).get(0)); + long entryId = ledgerHandle.addEntry(builder.build().toByteArray()); + rangeMarker.put((long) i, MLDataFormats.NestedPositionInfo + .newBuilder().setLedgerId(ledgerId).setEntryId(entryId).build()); + } + // init marker + c1.getRangeMarker().putAll(rangeMarker); + c1.setLastMarkDeleteEntry(new ManagedCursorImpl.MarkDeleteEntry( + new PositionImpl(1, 2), new HashMap<>(), null, null)); + // trigger switch and make sure it is finished + c1.startCreatingNewMetadataLedger(); + Awaitility.await().untilAsserted(() -> assertEquals(c1.getState(),"Open")); + + // validate marker + assertEquals(c1.getRangeMarker().size(), 3); + AtomicLong count = new AtomicLong(0); + long newLedgerId = ledgerId + 1; + c1.getRangeMarker().forEach((key, value) -> { + assertEquals(key.longValue(), count.get()); + assertEquals(value.getLedgerId(), newLedgerId); + assertEquals(value.getEntryId(), count.getAndIncrement()); + }); + + // validate entry + Enumeration entries = c1.getCursorLedgerHandle().readEntries(0, 3); + int counter = 0; + MLDataFormats.NestedPositionInfo.Builder positionBuilder = MLDataFormats.NestedPositionInfo.newBuilder(); + while (entries.hasMoreElements()) { + LedgerEntry entry = entries.nextElement(); + MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(entry.getEntryInputStream()); + System.out.println(positionInfo.getIndividualDeletedMessagesList()); + if (counter == 3) { + //marker + assertEquals(positionInfo.getMarkerIndexInfoCount(), 3); + AtomicLong subCounter = new AtomicLong(0); + positionInfo.getMarkerIndexInfoList().forEach(markerIndexInfo -> { + assertEquals(markerIndexInfo.getEntryPosition().getLedgerId(), newLedgerId); + assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), subCounter.getAndIncrement()); + }); + continue; + } else { + assertEquals(positionInfo.getMarkerIndexInfoCount(), 0); + } + MLDataFormats.MessageRange range = positionInfo.getIndividualDeletedMessagesList().get(0); + assertEquals(range.getLowerEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(0).build()); + assertEquals(range.getUpperEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(1).build()); + counter++; + } + + c1.close(); + ledger.close(); + } +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java index 53318ebc860d8..995ea73fc9ff3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java @@ -26,11 +26,14 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPair; import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer; +import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import java.util.ArrayList; import java.util.List; import java.util.Set; +import java.util.concurrent.TimeUnit; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; @@ -43,15 +46,18 @@ public class RangeSetWrapperTest { OrderedScheduler orderedScheduler; RangeSetWrapper set; ManagedLedgerConfig managedLedgerConfig; + ManagedCursorImpl managedCursor; @BeforeMethod public void setUp() { - managedLedger = mock(ManagedLedgerImpl.class); initManagedLedgerConfig(); + managedLedger = mock(ManagedLedgerImpl.class); + managedCursor = mock(ManagedCursorImpl.class); orderedScheduler = OrderedScheduler.newSchedulerBuilder() .numThreads(2).build(); doReturn(managedLedgerConfig).when(managedLedger).getConfig(); doReturn(orderedScheduler).when(managedLedger).getScheduledExecutor(); + doReturn(managedLedger).when(managedCursor).getManagedLedger(); } private void initManagedLedgerConfig() { @@ -79,7 +85,7 @@ public void testAddForSameKey() { } private void doTestAddForSameKey() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper(consumer, managedCursor); // add 0 to 5 set.addOpenClosed(0, 0, 0, 5); // add 8,9,10 @@ -99,7 +105,7 @@ private void doTestAddForSameKey() { @Test public void testAddForDifferentKey() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); // [98,100],[(1,5),(1,5)],[(1,10,1,15)],[(1,20),(1,20)],[(2,0),(2,10)] set.addOpenClosed(0, 98, 0, 99); set.addOpenClosed(0, 100, 1, 5); @@ -117,7 +123,7 @@ public void testAddForDifferentKey() { @Test public void testAddForDifferentKey2() { managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled(false); - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); // [98,100],[(1,5),(1,5)],[(1,10,1,15)],[(1,20),(1,20)],[(2,0),(2,10)] set.addOpenClosed(0, 98, 0, 99); set.addOpenClosed(0, 100, 1, 5); @@ -134,7 +140,7 @@ public void testAddForDifferentKey2() { @Test public void testAddCompareCompareWithGuava() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); // add 10K values for key 0 @@ -173,7 +179,7 @@ public void testAddCompareCompareWithGuava() { @Test public void testDeleteCompareWithGuava() throws Exception { - RangeSetWrapper set = new RangeSetWrapper<>(managedLedger, consumer); + RangeSetWrapper set = new RangeSetWrapper<>(consumer, managedCursor); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); // add 10K values for key 0 @@ -228,7 +234,7 @@ public void testDeleteCompareWithGuava() throws Exception { @Test public void testSpanWithGuava() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); set.addOpenClosed(0, 97, 0, 99); gSet.add(Range.openClosed(new LongPair(0, 97), new LongPair(0, 99))); @@ -253,7 +259,7 @@ public void testSpanWithGuava() { @Test public void testFirstRange() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); assertNull(set.firstRange()); set.addOpenClosed(0, 97, 0, 99); assertEquals(set.firstRange(), Range.openClosed(new LongPair(0, 97), new LongPair(0, 99))); @@ -268,7 +274,7 @@ public void testFirstRange() { @Test public void testLastRange() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); assertNull(set.lastRange()); Range range = Range.openClosed(new LongPair(0, 97), new LongPair(0, 99)); set.addOpenClosed(0, 97, 0, 99); @@ -289,7 +295,7 @@ public void testLastRange() { @Test public void testToString() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); set.addOpenClosed(0, 97, 0, 99); assertEquals(set.toString(), "[(0:97..0:99]]"); set.addOpenClosed(0, 98, 0, 105); @@ -300,7 +306,7 @@ public void testToString() { @Test public void testDeleteForDifferentKey() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); set.addOpenClosed(0, 97, 0, 99); set.addOpenClosed(0, 99, 1, 5); set.addOpenClosed(1, 9, 1, 15); @@ -331,7 +337,7 @@ public void testDeleteForDifferentKey() { @Test public void testDeleteWithAtMost() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); set.addOpenClosed(0, 98, 0, 99); set.addOpenClosed(0, 100, 1, 5); set.addOpenClosed(1, 10, 1, 15); @@ -357,7 +363,7 @@ public void testDeleteWithAtMost() { @Test public void testDeleteWithAtMost2() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); set.addOpenClosed(0, 98, 0, 99); set.addOpenClosed(0, 100, 1, 5); set.addOpenClosed(1, 10, 1, 15); @@ -377,7 +383,7 @@ public void testDeleteWithAtMost2() { assertEquals(ranges.get(count), (Range.openClosed(new LongPair(2, 25), new LongPair(2, 28)))); managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled(false); - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); set.addOpenClosed(0, 98, 0, 99); set.addOpenClosed(0, 100, 1, 5); set.addOpenClosed(1, 10, 1, 15); @@ -397,7 +403,7 @@ public void testDeleteWithAtMost2() { @Test public void testDeleteWithLeastMost() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); set.addOpenClosed(0, 98, 0, 99); set.addOpenClosed(0, 100, 1, 5); set.addOpenClosed(1, 10, 1, 15); @@ -425,7 +431,7 @@ public void testDeleteWithLeastMost() { @Test public void testRangeContaining() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); set.add(Range.closed(new LongPair(0, 98), new LongPair(0, 99))); set.add(Range.closed(new LongPair(0, 100), new LongPair(1, 5))); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); @@ -461,9 +467,48 @@ public void testRangeContaining() { assertEquals(set.rangeContaining(position.getKey(), position.getValue()), gSet.rangeContaining(position)); } + @Test + public void testLruCache() { + RangeSetWrapper.LruCache lruCache = + new RangeSetWrapper.LruCache<>(); + lruCache.touch(1); + lruCache.touch(2); + lruCache.touch(3); + lruCache.touch(4); + assertEquals(lruCache.getEldestEntry().getKey().intValue(), 1); + lruCache.touch(4); + lruCache.touch(1); + assertEquals(lruCache.getEldestEntry().getKey().intValue(), 2); + lruCache.touch(2); + assertEquals(lruCache.getEldestEntry().getKey().intValue(), 3); + lruCache.remove(3); + assertEquals(lruCache.getEldestEntry().getKey().intValue(), 4); + } + + @Test + public void testLruSchedule() { + LongPairConsumer positionRangeConverter = PositionImpl::new; + RangeSetWrapper wrapper = new RangeSetWrapper<>(positionRangeConverter, managedCursor); + wrapper.addOpenClosed(0, 1, 0, 99); + wrapper.addOpenClosed(0, 110, 0, 150); + wrapper.addOpenClosed(2, 1, 2, 99); + wrapper.addOpenClosed(2, 110, 2, 150); + wrapper.addOpenClosed(3, 1, 3, 99); + wrapper.addOpenClosed(3, 110, 3, 150); + // trigger touch + wrapper.contains(0, 20); + managedLedgerConfig.setMaxUnackedRangesInMemoryBytes(0); + Awaitility.await().atMost(20, TimeUnit.SECONDS).untilAsserted(() -> + assertEquals(wrapper.getRangeSet().size(), 2)); + assertEquals(wrapper.getRangeSet().toString(), "[(0:1..0:99],(0:110..0:150]]"); + assertEquals(wrapper.getLruCounter().getKeys().size(), 1); + List list = new ArrayList<>(wrapper.getLruCounter().getKeys()); + assertEquals(list.get(0).longValue(), 0L); + } + @Test public void testCacheFlagConflict() { - set = new RangeSetWrapper<>(managedLedger, consumer); + set = new RangeSetWrapper<>(consumer, managedCursor); set.add(Range.openClosed(new LongPair(0, 1), new LongPair(0, 2))); set.add(Range.openClosed(new LongPair(0, 3), new LongPair(0, 4))); assertEquals(set.toString(), "[(0:1..0:2],(0:3..0:4]]"); @@ -485,9 +530,7 @@ private List> getConnectedRange(Set> gRanges) { : previousUpper.getValue() - 1); int currentLowerValue = (int) (range.lowerBoundType().equals(BoundType.CLOSED) ? currentLower.getValue() : currentLower.getValue() + 1); - boolean connected = (previousUpper.getKey() == currentLower.getKey()) - ? (previousUpperValue >= currentLowerValue) - : false; + boolean connected = previousUpper.getKey() == currentLower.getKey() && (previousUpperValue >= currentLowerValue); if (connected) { lastRange = Range.closed(lastRange.lowerEndpoint(), range.upperEndpoint()); } else { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java index 34cb3280f9907..a8e1606160df2 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.common.util.collections; -import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.collect.BoundType; import com.google.common.collect.Range; import java.util.ArrayList; @@ -29,6 +28,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import static com.google.common.base.Preconditions.checkNotNull; /** * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of @@ -367,6 +367,12 @@ public void remove(Range range) { updatedAfterCachedForToString = true; } + public int getRangeBitSetMapMemorySize() { + AtomicInteger count = new AtomicInteger(0); + rangeBitSetMap.forEach((key, value) -> count.addAndGet(value.size() / 8 + 8)); + return count.get(); + } + private int getSafeEntry(LongPair position) { return (int) Math.max(position.getValue(), -1); } From 30c7e25c65d793ba997dc3de58f6cfdd315b05bd Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Sun, 23 May 2021 02:04:30 +0800 Subject: [PATCH 06/18] add unit test --- .../mledger/impl/ManagedCursorImpl.java | 159 +++++++++++----- .../mledger/impl/RangeSetWrapper.java | 64 ++++--- .../mledger/impl/MultiEntryPositionTest.java | 172 ++++++++++++++++-- 3 files changed, 316 insertions(+), 79 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index c69041995513f..6625e43c97d67 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -47,6 +47,7 @@ import java.util.Collections; import java.util.Enumeration; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -1817,6 +1818,7 @@ public void operationComplete() { try { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); + cleanRangeMarker(); if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null) { Map subMap = batchDeletedIndexes.subMap(PositionImpl.earliest, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true); subMap.values().forEach(BitSetRecyclable::recycle); @@ -1866,6 +1868,17 @@ public void operationFailed(ManagedLedgerException exception) { }); } + void cleanRangeMarker() { + if (rangeMarker != null && rangeMarker.size() > 0) { + long ledgerId = individualDeletedMessages.firstRange().lowerEndpoint().getLedgerId(); + rangeMarker.forEach((key, value) -> { + if (key < ledgerId) { + rangeMarker.remove(key); + } + }); + } + } + @Override public void delete(final Position position) throws InterruptedException, ManagedLedgerException { delete(Collections.singletonList(position)); @@ -2604,7 +2617,6 @@ private List buildIndividualDeletedMessageRanges() { if (individualDeletedMessages.isEmpty()) { return Collections.emptyList(); } - MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo .newBuilder(); MLDataFormats.MessageRange.Builder messageRangeBuilder = MLDataFormats.MessageRange.newBuilder(); @@ -2666,28 +2678,44 @@ void persistPositionToLedger(final LedgerHandle newLedgerHandle, MarkDeleteEntry } } - private void persistPositionToMultiEntry(LedgerHandle lh, MarkDeleteEntry mdEntry, VoidCallback callback) { + private synchronized void persistPositionToMultiEntry(LedgerHandle lh, MarkDeleteEntry mdEntry, VoidCallback callback) { checkNotNull(lh); - // get range info and group by ledger ID - Map> rangeGroupByLedgerId = getRangeGroupByLedgerId(); - Map> deletionIndexInfoGroupByLedgerId = - getDeletionIndexInfosGroupByLedgerId(); - List> callbacks = Collections.synchronizedList(new ArrayList<>()); - // build PositionInfo + RangeSetWrapper rangeSetWrapper; + Map clonedMarker; + Map> rangeGroupByLedgerId; + Map> deletionIndexInfoGroupByLedgerId; + lock.writeLock().lock(); + try { + rangeSetWrapper = (RangeSetWrapper) individualDeletedMessages; + Set dirtyRange = new HashSet<>(rangeSetWrapper.getDirtyKeyRecorder()); + clonedMarker = new ConcurrentHashMap<>(rangeMarker); + // get range info and group by ledger ID + rangeGroupByLedgerId = getRangeGroupByLedgerId(dirtyRange); + deletionIndexInfoGroupByLedgerId = getDeletionIndexInfosGroupByLedgerId(dirtyRange); + } finally { + lock.writeLock().unlock(); + } + List> callbacks = new ArrayList<>(); PositionInfo.Builder entryBuilder = getPositionBuilder(mdEntry).addProperties(createLruEntryTag()); // save entries to ledger for (Map.Entry> messageRanges : rangeGroupByLedgerId.entrySet()) { + // clean the data from previous round + if (entryBuilder.getBatchedEntryDeletionIndexInfoCount() > 0) { + entryBuilder.removeBatchedEntryDeletionIndexInfo(0); + } + if (entryBuilder.getIndividualDeletedMessagesCount() > 0) { + entryBuilder.removeIndividualDeletedMessages(0); + } CompletableFuture future = new CompletableFuture<>(); callbacks.add(future); - PositionInfo pi = entryBuilder.addAllIndividualDeletedMessages(messageRanges.getValue()) - .addAllBatchedEntryDeletionIndexInfo( - deletionIndexInfoGroupByLedgerId.get(messageRanges.getKey())) - .build(); + List deleteInfos = + deletionIndexInfoGroupByLedgerId.get(messageRanges.getKey()); + entryBuilder.addAllBatchedEntryDeletionIndexInfo(deleteInfos == null ? Collections.emptyList() : deleteInfos); + PositionInfo pi = entryBuilder.addAllIndividualDeletedMessages(messageRanges.getValue()).build(); lh.asyncAddEntry(pi.toByteArray(), (rc, lh1, entryId, ctx) -> { if (rc == BKException.Code.OK) { - rangeMarker.put(messageRanges.getKey(), + clonedMarker.put(messageRanges.getKey(), buildPositionInfo(lh1.getLedgerMetadata().getLedgerId(), entryId)); - mbean.persistToLedger(true); future.complete(null); } else { future.completeExceptionally(BKException.create(rc)); @@ -2696,15 +2724,18 @@ private void persistPositionToMultiEntry(LedgerHandle lh, MarkDeleteEntry mdEntr } // save marker FutureUtil.waitForAll(callbacks).thenCompose((x) -> { + mbean.persistToLedger(true); // no entry is saved, so marker is not dirty if (rangeGroupByLedgerId.size() < 1) { return CompletableFuture.completedFuture(null); } PositionInfo.Builder writerBuilder = getPositionBuilder(mdEntry).addProperties(createLruMarkerTag()); - writerBuilder.addAllMarkerIndexInfo(buildMarkerIndexMap(rangeMarker)); + writerBuilder.addAllMarkerIndexInfo(buildMarkerIndexMap(clonedMarker)); return saveMarker(lh, writerBuilder.build().toByteArray()); }).thenCompose((x) -> { - // callback and check whether should create new ledger + // entries and marker were successfully saved, update the rangeMarker cache + rangeMarker.putAll(clonedMarker); + // check whether should create a new ledger return checkIfNeedCreateNewLruLedgerAndSwitch(lh); }).thenAccept((newLedgerHandler) -> { callback.operationComplete(); @@ -2719,6 +2750,7 @@ private void persistPositionToMultiEntry(LedgerHandle lh, MarkDeleteEntry mdEntr (ManagedLedgerException) e : new ManagedLedgerException(e)); return null; }); + } private PositionInfo.Builder getPositionBuilder(MarkDeleteEntry mdEntry) { @@ -2758,10 +2790,10 @@ private CompletableFuture checkIfNeedCreateNewLruLedgerAndSwitch(L private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle, LedgerHandle newHandle) { CompletableFuture result = new CompletableFuture<>(); // copy all entries in marker to new ledger - List> futures = Collections.synchronizedList(new ArrayList<>()); - Map dirtyLedgerRecorder = new ConcurrentHashMap<>(); + List> futures = new ArrayList<>(); + Map clonedMap = new ConcurrentHashMap<>(rangeMarker); try { - for (Map.Entry longPositionEntry : rangeMarker.entrySet()) { + for (Map.Entry longPositionEntry : clonedMap.entrySet()) { MLDataFormats.NestedPositionInfo position = longPositionEntry.getValue(); Enumeration entryEnumeration = oldHandle.readEntries(position.getEntryId(), position.getEntryId()); @@ -2777,7 +2809,7 @@ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle if (rc != BKException.Code.OK) { future.completeExceptionally(BKException.create(rc)); } else { - dirtyLedgerRecorder.put(longPositionEntry.getKey(), + clonedMap.put(longPositionEntry.getKey(), buildPositionInfo(newHandle.getLedgerMetadata().getLedgerId(), entryId)); future.complete(null); } @@ -2793,8 +2825,6 @@ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle result.completeExceptionally(e); return; } - Map clonedMap = Maps.newHashMap(rangeMarker); - clonedMap.putAll(dirtyLedgerRecorder); PositionInfo.Builder writerBuilder = getPositionBuilder(lastMarkDeleteEntry) .addProperties(createLruMarkerTag()).addAllMarkerIndexInfo(buildMarkerIndexMap(clonedMap)); saveMarker(newHandle, writerBuilder.build().toByteArray()).whenComplete((re, ex) -> { @@ -2834,29 +2864,74 @@ private Optional getLastAvailableMarker(LedgerHandle ledgerHandle) return Optional.empty(); } - Map> getDeletionIndexInfosGroupByLedgerId() { - List batchDeletionIndexInfos - = buildBatchEntryDeletionIndexInfoList(); - Map> ledgerIdToIndexInfo = new HashMap<>(); - batchDeletionIndexInfos.forEach(indexInfo -> { - long ledgerId = indexInfo.getPosition().getLedgerId(); - List list = ledgerIdToIndexInfo - .computeIfAbsent(ledgerId, (le) -> new ArrayList<>()); - list.add(indexInfo); - }); + Map> getDeletionIndexInfosGroupByLedgerId( + Set dirtyRange) { + if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes == null || batchDeletedIndexes.isEmpty()) { + return Collections.emptyMap(); + } + MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo + .newBuilder(); + MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchedEntryDeletionIndexInfo + .newBuilder(); + Map> ledgerIdToIndexInfo = + Maps.newHashMapWithExpectedSize(batchDeletedIndexes.size()); + Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + while (iterator.hasNext() && ledgerIdToIndexInfo.size() < config.getMaxBatchDeletedIndexToPersist()) { + Map.Entry entry = iterator.next(); + if (dirtyRange != null && !dirtyRange.contains(entry.getKey().getLedgerId())) { + continue; + } + nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); + nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); + batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); + long[] array = entry.getValue().toLongArray(); + List deleteSet = new ArrayList<>(array.length); + for (long l : array) { + deleteSet.add(l); + } + batchDeletedIndexInfoBuilder.clearDeleteSet(); + batchDeletedIndexInfoBuilder.addAllDeleteSet(deleteSet); + List list = ledgerIdToIndexInfo.computeIfAbsent( + entry.getKey().getLedgerId(), (le) -> new ArrayList<>()); + list.add(batchDeletedIndexInfoBuilder.build()); + } return ledgerIdToIndexInfo; } - Map> getRangeGroupByLedgerId() { - List rangeList = buildIndividualDeletedMessageRanges(); - Map> ledgerIdToMessageRange = new HashMap<>(); - rangeList.forEach(messageRange -> { - long ledgerId = messageRange.getLowerEndpoint().getLedgerId(); - List list = ledgerIdToMessageRange - .computeIfAbsent(ledgerId, (le) -> new ArrayList<>()); - list.add(messageRange); - }); - return ledgerIdToMessageRange; + Map> getRangeGroupByLedgerId(Set dirtyRange) { + lock.readLock().lock(); + try { + if (individualDeletedMessages.isEmpty()) { + return new HashMap<>(); + } + Map> ledgerIdToMessageRange = + Maps.newHashMapWithExpectedSize(individualDeletedMessages.size()); + MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo + .newBuilder(); + MLDataFormats.MessageRange.Builder messageRangeBuilder = MLDataFormats.MessageRange.newBuilder(); + AtomicInteger acksSerializedSize = new AtomicInteger(0); + individualDeletedMessages.forEach((positionRange) -> { + if (dirtyRange != null && !dirtyRange.contains(positionRange.upperEndpoint().ledgerId)) { + return true; + } + nestedPositionBuilder.setLedgerId(positionRange.lowerEndpoint().ledgerId) + .setEntryId(positionRange.lowerEndpoint().entryId); + messageRangeBuilder.setLowerEndpoint(nestedPositionBuilder.build()); + nestedPositionBuilder.setLedgerId(positionRange.upperEndpoint().ledgerId) + .setEntryId(positionRange.upperEndpoint().entryId); + messageRangeBuilder.setUpperEndpoint(nestedPositionBuilder.build()); + MessageRange messageRange = messageRangeBuilder.build(); + acksSerializedSize.addAndGet(messageRange.getSerializedSize()); + List list = ledgerIdToMessageRange + .computeIfAbsent(messageRange.getUpperEndpoint().getLedgerId(), (le) -> new ArrayList<>()); + list.add(messageRange); + return ledgerIdToMessageRange.size() <= config.getMaxUnackedRangesToPersist(); + }); + this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); + return ledgerIdToMessageRange; + } finally { + lock.readLock().unlock(); + } } private void persistPositionToSingleEntry(LedgerHandle lh, MarkDeleteEntry mdEntry, VoidCallback callback) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index c40d287ab0434..45bbb80678dfb 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -31,11 +31,14 @@ import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Enumeration; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -51,6 +54,7 @@ public class RangeSetWrapper> implements LongPairRangeSe private final ScheduledFuture future; private final OrderedScheduler scheduler; private final LruCache lruCounter = new LruCache<>(); + private final Set dirtyKeyRecorder = Collections.newSetFromMap(new ConcurrentHashMap<>()); public RangeSetWrapper(LongPairConsumer rangeConverter, ManagedCursorImpl managedCursor) { checkNotNull(managedCursor); @@ -74,6 +78,7 @@ public RangeSetWrapper(LongPairConsumer rangeConverter, ManagedCursorImpl man @Override public void addOpenClosed(long lowerKey, long lowerValue, long upperKey, long upperValue) { lruTouch(upperKey); + dirtyKeyRecorder.add(upperKey); rangeSet.addOpenClosed(lowerKey, lowerValue, upperKey, upperValue); } @@ -225,6 +230,10 @@ public void close() throws Exception { } } + public Set getDirtyKeyRecorder() { + return dirtyKeyRecorder; + } + @Override public String toString() { return rangeSet.toString(); @@ -234,32 +243,39 @@ class LruTask implements Runnable { @Override public void run() { - log.debug("start schedule LruTask"); - // remove invalid key which is removed in rangeSet - Range span = rangeSet.span(); - for (Long ledgerId : lruCounter.getKeys()) { - if (span.lowerEndpoint().compareTo(rangeConverter.apply(ledgerId, Integer.MAX_VALUE - 1)) > 0) { - lruCounter.remove(ledgerId); - log.info("LruTask remove invalid key {}", ledgerId); - } else { - break; + try { + log.debug("start schedule LruTask, keys in cache:{}", lruCounter.getKeys()); + // remove invalid key which is removed in rangeSet + Range firstRange = rangeSet.firstRange(); + // use Iterator to avoid ConcurrentModifyException + Iterator iterator = lruCounter.getKeys().iterator(); + while (iterator.hasNext()) { + long ledgerId = iterator.next(); + if (firstRange.lowerEndpoint().compareTo(rangeConverter.apply(ledgerId, Integer.MAX_VALUE - 1)) > 0) { + iterator.remove(); + log.info("LruTask remove invalid key {}", ledgerId); + } else { + break; + } } - } - if (isReachLruSwitchThreshold() && lruCounter.size() > 1) { - long eldestKey = lruCounter.removeEldestEntryAndGet(); - log.info("Reach switching threshold, try to remove eldest ledger {}, range size {}", - eldestKey, rangeSet.size()); - if (rangeSet instanceof ConcurrentOpenLongPairRangeSet) { - ConcurrentOpenLongPairRangeSet set = - (ConcurrentOpenLongPairRangeSet) rangeSet; - set.remove(Range.openClosed(new LongPair(eldestKey, 0), new LongPair(eldestKey, - Integer.MAX_VALUE - 1))); - } else { - LongPairRangeSet.DefaultRangeSet set = - (LongPairRangeSet.DefaultRangeSet) rangeSet; - set.remove(Range.openClosed(rangeConverter.apply(eldestKey, 0), - rangeConverter.apply(eldestKey, Integer.MAX_VALUE - 1))); + if (isReachLruSwitchThreshold() && lruCounter.size() > 1) { + long eldestKey = lruCounter.removeEldestEntryAndGet(); + log.info("Reach switching threshold, try to remove eldest ledger {}, range size {}", + eldestKey, rangeSet.size()); + if (rangeSet instanceof ConcurrentOpenLongPairRangeSet) { + ConcurrentOpenLongPairRangeSet set = + (ConcurrentOpenLongPairRangeSet) rangeSet; + set.remove(Range.openClosed(new LongPair(eldestKey, 0), new LongPair(eldestKey, + Integer.MAX_VALUE - 1))); + } else { + LongPairRangeSet.DefaultRangeSet set = + (LongPairRangeSet.DefaultRangeSet) rangeSet; + set.remove(Range.openClosed(rangeConverter.apply(eldestKey, 0), + rangeConverter.apply(eldestKey, Integer.MAX_VALUE - 1))); + } } + } catch (Exception e) { + log.error("LruTask run failed", e); } } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java index c7af0f96d0792..6f2f127426e91 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java @@ -20,8 +20,10 @@ import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.common.util.collections.BitSetRecyclable; @@ -30,10 +32,15 @@ import java.util.Arrays; import java.util.Enumeration; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; public class MultiEntryPositionTest extends MockedBookKeeperTestCase { @@ -48,7 +55,7 @@ public void testGetRangeGroupByLedgerId() throws Exception { c1.getIndividuallyDeletedMessagesSet().addOpenClosed(1, 0, 1, 1); c1.getIndividuallyDeletedMessagesSet().addOpenClosed(2, 0, 2, 1); - Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(); + Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(null); assertEquals(rangeGroupByLedgerId.size(), 3); AtomicLong count = new AtomicLong(0); @@ -61,6 +68,11 @@ public void testGetRangeGroupByLedgerId() throws Exception { assertEquals(value.get(0).getUpperEndpoint().getEntryId(), 1); }); + Set filter = new HashSet<>(); + filter.add(2L); + rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(filter); + assertEquals(rangeGroupByLedgerId.size(), 1); + assertTrue(rangeGroupByLedgerId.containsKey(2L)); c1.close(); ledger.close(); } @@ -79,7 +91,8 @@ public void testGetDeletionIndexInfosGroupByLedgerId() throws Exception { c1.getBatchDeletedIndexes().put(new PositionImpl(1,0), list.get(1)); c1.getBatchDeletedIndexes().put(new PositionImpl(2,0), list.get(2)); - Map> map = c1.getDeletionIndexInfosGroupByLedgerId(); + Map> map = c1. + getDeletionIndexInfosGroupByLedgerId(null); assertEquals(map.size(), 3); AtomicLong count = new AtomicLong(0); @@ -90,16 +103,29 @@ public void testGetDeletionIndexInfosGroupByLedgerId() throws Exception { assertEquals(value.get(0).getPosition().getEntryId(), 0); }); + Set filter = new HashSet<>(); + filter.add(2L); + map = c1.getDeletionIndexInfosGroupByLedgerId(filter); + assertEquals(map.size(), 1); + assertTrue(map.containsKey(2L)); + c1.close(); ledger.close(); list.forEach(BitSetRecyclable::recycle); } + /** + * Covered chain: internalAsyncMarkDelete & NoLedger -> internalAsyncMarkDelete + NoLedger -> + * startCreatingNewMetadataLedger -> createNewMetadataLedgerAndSwitch -> doCreateNewMetadataLedger -> + * persistPositionToLedger -> switchToNewLedger -> flushPendingMarkDeletes -> internalMarkDelete -> + * persistPositionToLedger -> remove individualDeletedMessages + * @throws Exception + */ @Test public void testCopyLruEntriesToNewLedger() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setEnableLruCacheMaxUnackedRanges(true); - ManagedLedger ledger = factory.open("my_test_ledger", config); + ManagedLedger ledger = factory.open("my_test_ledger" + UUID.randomUUID(), config); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); //init new ledger c1.startCreatingNewMetadataLedger(); @@ -110,7 +136,7 @@ public void testCopyLruEntriesToNewLedger() throws Exception { c1.getIndividuallyDeletedMessagesSet().addOpenClosed(0, 0, 0, 1); c1.getIndividuallyDeletedMessagesSet().addOpenClosed(1, 0, 1, 1); c1.getIndividuallyDeletedMessagesSet().addOpenClosed(2, 0, 2, 1); - Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(); + Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(null); MLDataFormats.PositionInfo.Builder builder = MLDataFormats.PositionInfo.newBuilder().setLedgerId(1).setEntryId(0); Map rangeMarker = new HashMap<>(); @@ -126,37 +152,82 @@ public void testCopyLruEntriesToNewLedger() throws Exception { // init marker c1.getRangeMarker().putAll(rangeMarker); c1.setLastMarkDeleteEntry(new ManagedCursorImpl.MarkDeleteEntry( - new PositionImpl(1, 2), new HashMap<>(), null, null)); + new PositionImpl(0, 0), new HashMap<>(), null, null)); + // init PendingMarkDeletes + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(3, 0, 3, 1); + CompletableFuture future = new CompletableFuture<>(); + c1.pendingMarkDeleteOps.add(new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(0, 0), + new HashMap<>(), new AsyncCallbacks.MarkDeleteCallback() { + @Override + public void markDeleteComplete(Object ctx) { + future.complete(null); + } + + @Override + public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null)); // trigger switch and make sure it is finished c1.startCreatingNewMetadataLedger(); Awaitility.await().untilAsserted(() -> assertEquals(c1.getState(),"Open")); - + future.get(); // validate marker - assertEquals(c1.getRangeMarker().size(), 3); - AtomicLong count = new AtomicLong(0); + assertEquals(c1.getRangeMarker().size(), 4); + // After copying, flushPendingMarkDeletes will be triggered. + // Copying will occupy entryId 0-3, and flushPendingMarkDeletes will occupy 4-7. + AtomicLong entryIdCounter = new AtomicLong(4); + AtomicLong ledgerIdCounter = new AtomicLong(0); long newLedgerId = ledgerId + 1; c1.getRangeMarker().forEach((key, value) -> { - assertEquals(key.longValue(), count.get()); + // in IndividuallyDeletedMessages ledgerId is from 0 to 3 + assertEquals(key.longValue(), ledgerIdCounter.getAndIncrement()); assertEquals(value.getLedgerId(), newLedgerId); - assertEquals(value.getEntryId(), count.getAndIncrement()); + assertEquals(value.getEntryId(), entryIdCounter.getAndIncrement()); }); - // validate entry + // Verify the copied entry and marker Enumeration entries = c1.getCursorLedgerHandle().readEntries(0, 3); int counter = 0; MLDataFormats.NestedPositionInfo.Builder positionBuilder = MLDataFormats.NestedPositionInfo.newBuilder(); while (entries.hasMoreElements()) { LedgerEntry entry = entries.nextElement(); MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(entry.getEntryInputStream()); - System.out.println(positionInfo.getIndividualDeletedMessagesList()); if (counter == 3) { - //marker + //the last entry is marker + assertEquals(positionInfo.getMarkerIndexInfoCount(), 3); + AtomicLong subCounter = new AtomicLong(0); + positionInfo.getMarkerIndexInfoList().forEach(markerIndexInfo -> { + assertEquals(markerIndexInfo.getEntryPosition().getLedgerId(), newLedgerId); + assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), subCounter.getAndIncrement()); + }); + assertEquals(subCounter.get(), 3); + continue; + } else { + assertEquals(positionInfo.getMarkerIndexInfoCount(), 0); + } + MLDataFormats.MessageRange range = positionInfo.getIndividualDeletedMessagesList().get(0); + assertEquals(range.getLowerEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(0).build()); + assertEquals(range.getUpperEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(1).build()); + counter++; + } + assertEquals(counter, 3); + + // Verify entries and marker created by flushPendingMarkDeletes + entries = c1.getCursorLedgerHandle().readEntries(4, 7); + counter = 0; + while (entries.hasMoreElements()) { + LedgerEntry entry = entries.nextElement(); + MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(entry.getEntryInputStream()); + if (counter == 4) { + // the last one is marker assertEquals(positionInfo.getMarkerIndexInfoCount(), 3); AtomicLong subCounter = new AtomicLong(0); positionInfo.getMarkerIndexInfoList().forEach(markerIndexInfo -> { assertEquals(markerIndexInfo.getEntryPosition().getLedgerId(), newLedgerId); assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), subCounter.getAndIncrement()); }); + assertEquals(subCounter.get(), 4); continue; } else { assertEquals(positionInfo.getMarkerIndexInfoCount(), 0); @@ -166,8 +237,83 @@ public void testCopyLruEntriesToNewLedger() throws Exception { assertEquals(range.getUpperEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(1).build()); counter++; } + assertEquals(counter, 4); + // Verify cache + assertEquals(c1.getIndividuallyDeletedMessages(), "[(0:0..0:1],(1:0..1:1],(2:0..2:1],(3:0..3:1]]"); + assertEquals(c1.getRangeMarker().size(), 4); + RangeSetWrapper setWrapper = (RangeSetWrapper) c1.getIndividuallyDeletedMessagesSet(); + Awaitility.await().untilAsserted(() -> assertEquals(setWrapper.getLruCounter().size(), 4)); c1.close(); ledger.close(); } + + /** + * Covered chain: internalAsyncMarkDelete & Ledger is Open -> internalMarkDelete -> persistPositionToLedger + * -> remove individualDeletedMessages + * @throws Exception + */ + @Test + public void testPersistPositionToLedger() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setEnableLruCacheMaxUnackedRanges(true); + ManagedLedger ledger = factory.open("my_test_ledger" + UUID.randomUUID(), config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + //init new ledger + c1.startCreatingNewMetadataLedger(); + Awaitility.await().untilAsserted(() -> assertEquals(c1.getState(),"Open")); + LedgerHandle ledgerHandle = c1.getCursorLedgerHandle(); + long ledgerId = ledgerHandle.getId(); + // init IndividuallyDeletedMessages + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(0, 0, 0, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(1, 0, 1, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(2, 0, 2, 1); + CompletableFuture future = new CompletableFuture<>(); + c1.internalMarkDelete(new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(1, 10), new HashMap<>(), + new AsyncCallbacks.MarkDeleteCallback() { + @Override + public void markDeleteComplete(Object ctx) { + future.complete(null); + } + + @Override + public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null)); + + future.get(); + Map marker = c1.getRangeMarker(); + // markDelete position is (1, 10] , so only [(2:0..2:1]] is left + assertEquals(marker.size(), 1); + MLDataFormats.NestedPositionInfo positionInfo = marker.get((long) 2); + assertEquals(positionInfo.getLedgerId(), ledgerId); + assertEquals(positionInfo.getEntryId(), 3); + assertEquals(c1.getIndividuallyDeletedMessages(), "[(2:0..2:1]]"); + Enumeration entries = c1.getCursorLedgerHandle().readEntries(positionInfo.getEntryId() - 2, + positionInfo.getEntryId() + 1); + int counter = 0; + MLDataFormats.NestedPositionInfo.Builder positionBuilder = MLDataFormats.NestedPositionInfo.newBuilder(); + while (entries.hasMoreElements()) { + LedgerEntry entry = entries.nextElement(); + MLDataFormats.PositionInfo position = MLDataFormats.PositionInfo.parseFrom(entry.getEntryInputStream()); + if (counter == 3) { + // the last one is marker + assertEquals(position.getMarkerIndexInfoCount(), 3); + AtomicLong subCounter = new AtomicLong(positionInfo.getEntryId() - 2); + position.getMarkerIndexInfoList().forEach(markerIndexInfo -> { + assertEquals(markerIndexInfo.getEntryPosition().getLedgerId(), ledgerId); + assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), subCounter.getAndIncrement()); + }); + assertEquals(subCounter.get(), 4); + continue; + } else { + assertEquals(position.getMarkerIndexInfoCount(), 0); + } + MLDataFormats.MessageRange range = position.getIndividualDeletedMessagesList().get(0); + assertEquals(range.getLowerEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(0).build()); + assertEquals(range.getUpperEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(1).build()); + counter++; + } + } } From 9f563f905df7d0fd2bf1177d8188b160f714d3b9 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Sun, 23 May 2021 20:27:32 +0800 Subject: [PATCH 07/18] add unit test --- .../mledger/impl/ManagedCursorImpl.java | 134 +++++++----- .../mledger/impl/RangeSetWrapper.java | 14 ++ .../mledger/impl/MultiEntryPositionTest.java | 192 +++++++++++++++++- 3 files changed, 288 insertions(+), 52 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 6625e43c97d67..8071c9e0e8a7f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -349,7 +349,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } } - private void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback) { + void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback) { ledger.mbean.startCursorLedgerOpenOp(); long ledgerId = info.getCursorsLedgerId(); OpenCallback openCallback = (rc, lh, ctx) -> { @@ -364,13 +364,12 @@ private void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback } PositionInfo markerPosition = optional.get(); try { + rangeMarker.clear(); for (MLDataFormats.MarkerIndexInfo markerIndexInfo : markerPosition.getMarkerIndexInfoList()) { MLDataFormats.NestedPositionInfo nestedPositionInfo = markerIndexInfo.getEntryPosition(); - Enumeration entryEnumeration = lh.readEntries(nestedPositionInfo.getEntryId(), - nestedPositionInfo.getEntryId()); - if (entryEnumeration.hasMoreElements()) { - LedgerEntry entry = entryEnumeration.nextElement(); - PositionInfo entryPosition = PositionInfo.parseDelimitedFrom(entry.getEntryInputStream()); + LedgerEntry ledgerEntry = readEntries(lh, nestedPositionInfo.getEntryId()); + if (ledgerEntry != null) { + PositionInfo entryPosition = PositionInfo.parseDelimitedFrom(ledgerEntry.getEntryInputStream()); if (entryPosition.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(entryPosition.getIndividualDeletedMessagesList(), false); } @@ -378,6 +377,7 @@ private void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback && entryPosition.getBatchedEntryDeletionIndexInfoCount() > 0) { recoverBatchDeletedIndexes(entryPosition.getBatchedEntryDeletionIndexInfoList(), false); } + rangeMarker.put(markerIndexInfo.getTargetLedgerId(), nestedPositionInfo); } if (individualDeletedMessages.size() > config.getMaxUnackedRangesInMemoryBytes()) { break; @@ -392,9 +392,11 @@ private void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback }; try { bookkeeper.asyncOpenLedger(ledgerId, digestType, config.getPassword(), openCallback, null); + ledger.mbean.endCursorLedgerOpenOp(); } catch (Throwable t) { log.error("[{}] Encountered error on opening cursor ledger {} for cursor {}", ledger.getName(), ledgerId, name, t); + ledger.mbean.endCursorLedgerOpenOp(); openCallback.openComplete(BKException.Code.UnexpectedConditionException, null, null); } } @@ -613,6 +615,25 @@ public void operationFailed(ManagedLedgerException exception) { }); } + public LedgerEntry readEntries(LedgerHandle ledgerHandle, long entryId) { + CompletableFuture future = new CompletableFuture<>(); + ledgerHandle.asyncReadEntries(entryId, entryId, (rc1, lh1, seq, ctx1) -> { + if (rc1 != BKException.Code.OK) { + future.completeExceptionally(BKException.create(rc1)); + } + if (seq.hasMoreElements()) { + future.complete(seq.nextElement()); + } else { + future.complete(null); + } + }, null); + try { + return future.get(config.getMetadataOperationsTimeoutSeconds(), TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + @Override public List readEntries(int numberOfEntriesToRead) throws InterruptedException, ManagedLedgerException { checkArgument(numberOfEntriesToRead > 0); @@ -1818,12 +1839,12 @@ public void operationComplete() { try { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); - cleanRangeMarker(); if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null) { Map subMap = batchDeletedIndexes.subMap(PositionImpl.earliest, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true); subMap.values().forEach(BitSetRecyclable::recycle); subMap.clear(); } + cleanRangeMarker(); } finally { lock.writeLock().unlock(); } @@ -2588,9 +2609,9 @@ private MLDataFormats.NestedPositionInfo buildPositionInfo(long ledgerId, long e private List buildMarkerIndexMap( Map rangeMarker) { MLDataFormats.MarkerIndexInfo.Builder builder = MLDataFormats.MarkerIndexInfo.newBuilder(); - return rangeMarker.entrySet().stream().map((entry) -> - builder.setTargetLedgerId(entry.getKey()).setEntryPosition(entry.getValue()).build()) - .collect(Collectors.toList()); + return rangeMarker.entrySet().stream() + .map((entry) -> builder.setTargetLedgerId(entry.getKey()).setEntryPosition(entry.getValue()) + .build()).collect(Collectors.toList()); } private Map buildPropertiesMap(List longProperties) { @@ -2680,20 +2701,19 @@ void persistPositionToLedger(final LedgerHandle newLedgerHandle, MarkDeleteEntry private synchronized void persistPositionToMultiEntry(LedgerHandle lh, MarkDeleteEntry mdEntry, VoidCallback callback) { checkNotNull(lh); - RangeSetWrapper rangeSetWrapper; + RangeSetWrapper rangeSetWrapper = (RangeSetWrapper) individualDeletedMessages; + Set dirtyRange = new HashSet<>(rangeSetWrapper.getDirtyKeyRecords()); Map clonedMarker; Map> rangeGroupByLedgerId; Map> deletionIndexInfoGroupByLedgerId; - lock.writeLock().lock(); + lock.readLock().lock(); try { - rangeSetWrapper = (RangeSetWrapper) individualDeletedMessages; - Set dirtyRange = new HashSet<>(rangeSetWrapper.getDirtyKeyRecorder()); clonedMarker = new ConcurrentHashMap<>(rangeMarker); // get range info and group by ledger ID rangeGroupByLedgerId = getRangeGroupByLedgerId(dirtyRange); deletionIndexInfoGroupByLedgerId = getDeletionIndexInfosGroupByLedgerId(dirtyRange); } finally { - lock.writeLock().unlock(); + lock.readLock().unlock(); } List> callbacks = new ArrayList<>(); PositionInfo.Builder entryBuilder = getPositionBuilder(mdEntry).addProperties(createLruEntryTag()); @@ -2724,7 +2744,6 @@ private synchronized void persistPositionToMultiEntry(LedgerHandle lh, MarkDelet } // save marker FutureUtil.waitForAll(callbacks).thenCompose((x) -> { - mbean.persistToLedger(true); // no entry is saved, so marker is not dirty if (rangeGroupByLedgerId.size() < 1) { return CompletableFuture.completedFuture(null); @@ -2733,6 +2752,7 @@ private synchronized void persistPositionToMultiEntry(LedgerHandle lh, MarkDelet writerBuilder.addAllMarkerIndexInfo(buildMarkerIndexMap(clonedMarker)); return saveMarker(lh, writerBuilder.build().toByteArray()); }).thenCompose((x) -> { + mbean.persistToLedger(true); // entries and marker were successfully saved, update the rangeMarker cache rangeMarker.putAll(clonedMarker); // check whether should create a new ledger @@ -2744,6 +2764,7 @@ private synchronized void persistPositionToMultiEntry(LedgerHandle lh, MarkDelet }).exceptionally(e -> { log.warn("[{}] Error updating cursor {} position {} in meta-ledger {}", ledger.getName(), name, mdEntry.newPosition, lh.getId(), e); + rangeSetWrapper.getDirtyKeyRecorder().addAll(dirtyRange); STATE_UPDATER.compareAndSet(ManagedCursorImpl.this, State.Open, State.NoLedger); mbean.persistToLedger(false); callback.operationFailed(e instanceof ManagedLedgerException ? @@ -2787,6 +2808,12 @@ private CompletableFuture checkIfNeedCreateNewLruLedgerAndSwitch(L return future; } + /** + * + * @param oldHandle + * @param newHandle + * @return + */ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle, LedgerHandle newHandle) { CompletableFuture result = new CompletableFuture<>(); // copy all entries in marker to new ledger @@ -2826,7 +2853,8 @@ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle return; } PositionInfo.Builder writerBuilder = getPositionBuilder(lastMarkDeleteEntry) - .addProperties(createLruMarkerTag()).addAllMarkerIndexInfo(buildMarkerIndexMap(clonedMap)); + .addProperties(createLruMarkerTag()) + .addAllMarkerIndexInfo(buildMarkerIndexMap(clonedMap)); saveMarker(newHandle, writerBuilder.build().toByteArray()).whenComplete((re, ex) -> { if (ex != null) { result.completeExceptionally(ex); @@ -2839,18 +2867,17 @@ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle return result; } - private Optional getLastAvailableMarker(LedgerHandle ledgerHandle) { + Optional getLastAvailableMarker(LedgerHandle ledgerHandle) { long entryId = ledgerHandle.getLastAddConfirmed(); try { for (long i = entryId; i >= 0; i--) { - Enumeration entryEnumeration = ledgerHandle.readEntries(i, i); - if (!entryEnumeration.hasMoreElements()) { + LedgerEntry entry = readEntries(ledgerHandle, i); + if (entry == null) { return Optional.empty(); } - LedgerEntry ledgerEntry = entryEnumeration.nextElement(); - PositionInfo positionInfo = PositionInfo.parseFrom(ledgerEntry.getEntry()); + PositionInfo positionInfo = PositionInfo.parseFrom(entry.getEntryInputStream()); Map propertiesMap = buildPropertiesMap(positionInfo.getPropertiesList()); - if(!propertiesMap.containsKey(LRU_ENTRY) && propertiesMap.containsKey(LRU_MARKER)){ + if (!propertiesMap.containsKey(LRU_ENTRY) && !propertiesMap.containsKey(LRU_MARKER)) { log.info("Currently not using lru mode"); return Optional.empty(); } @@ -2866,36 +2893,41 @@ private Optional getLastAvailableMarker(LedgerHandle ledgerHandle) Map> getDeletionIndexInfosGroupByLedgerId( Set dirtyRange) { - if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes == null || batchDeletedIndexes.isEmpty()) { - return Collections.emptyMap(); - } - MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo - .newBuilder(); - MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchedEntryDeletionIndexInfo - .newBuilder(); - Map> ledgerIdToIndexInfo = - Maps.newHashMapWithExpectedSize(batchDeletedIndexes.size()); - Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); - while (iterator.hasNext() && ledgerIdToIndexInfo.size() < config.getMaxBatchDeletedIndexToPersist()) { - Map.Entry entry = iterator.next(); - if (dirtyRange != null && !dirtyRange.contains(entry.getKey().getLedgerId())) { - continue; - } - nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); - nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); - batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); - long[] array = entry.getValue().toLongArray(); - List deleteSet = new ArrayList<>(array.length); - for (long l : array) { - deleteSet.add(l); + lock.readLock().lock(); + try { + if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes == null || batchDeletedIndexes.isEmpty()) { + return Collections.emptyMap(); } - batchDeletedIndexInfoBuilder.clearDeleteSet(); - batchDeletedIndexInfoBuilder.addAllDeleteSet(deleteSet); - List list = ledgerIdToIndexInfo.computeIfAbsent( - entry.getKey().getLedgerId(), (le) -> new ArrayList<>()); - list.add(batchDeletedIndexInfoBuilder.build()); + MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo + .newBuilder(); + MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchedEntryDeletionIndexInfo + .newBuilder(); + Map> ledgerIdToIndexInfo = + Maps.newHashMapWithExpectedSize(batchDeletedIndexes.size()); + Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + while (iterator.hasNext() && ledgerIdToIndexInfo.size() < config.getMaxBatchDeletedIndexToPersist()) { + Map.Entry entry = iterator.next(); + if (dirtyRange != null && !dirtyRange.contains(entry.getKey().getLedgerId())) { + continue; + } + nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); + nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); + batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); + long[] array = entry.getValue().toLongArray(); + List deleteSet = new ArrayList<>(array.length); + for (long l : array) { + deleteSet.add(l); + } + batchDeletedIndexInfoBuilder.clearDeleteSet(); + batchDeletedIndexInfoBuilder.addAllDeleteSet(deleteSet); + List list = ledgerIdToIndexInfo.computeIfAbsent( + entry.getKey().getLedgerId(), (le) -> new ArrayList<>()); + list.add(batchDeletedIndexInfoBuilder.build()); + } + return ledgerIdToIndexInfo; + } finally { + lock.readLock().unlock(); } - return ledgerIdToIndexInfo; } Map> getRangeGroupByLedgerId(Set dirtyRange) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index 45bbb80678dfb..aa544e105d417 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -33,6 +33,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Enumeration; +import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -230,6 +231,12 @@ public void close() throws Exception { } } + public Set getDirtyKeyRecords() { + Set set = new HashSet<>(dirtyKeyRecorder); + dirtyKeyRecorder.removeAll(set); + return set; + } + public Set getDirtyKeyRecorder() { return dirtyKeyRecorder; } @@ -247,6 +254,9 @@ public void run() { log.debug("start schedule LruTask, keys in cache:{}", lruCounter.getKeys()); // remove invalid key which is removed in rangeSet Range firstRange = rangeSet.firstRange(); + if (firstRange == null) { + return; + } // use Iterator to avoid ConcurrentModifyException Iterator iterator = lruCounter.getKeys().iterator(); while (iterator.hasNext()) { @@ -309,6 +319,10 @@ public K removeEldestEntryAndGet() { return null; } + public void clear() { + linkedHashMap.clear(); + } + public Map.Entry getEldestEntry() { try { if (field == null) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java index 6f2f127426e91..bd090993cb4b2 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java @@ -27,18 +27,28 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.common.util.collections.BitSetRecyclable; +import org.apache.pulsar.metadata.api.Stat; import org.awaitility.Awaitility; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import org.testng.annotations.Test; import java.util.Arrays; +import java.util.Collections; import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertTrue; @@ -201,7 +211,6 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { assertEquals(markerIndexInfo.getEntryPosition().getLedgerId(), newLedgerId); assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), subCounter.getAndIncrement()); }); - assertEquals(subCounter.get(), 3); continue; } else { assertEquals(positionInfo.getMarkerIndexInfoCount(), 0); @@ -316,4 +325,185 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { counter++; } } + + /** + * Covered chain: initialize -> createNewMetadataLedgerAndSwitch -> doCreateNewMetadataLedger -> switchToNewLedger + * @throws Exception + */ + @Test + public void testInitialize() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setEnableLruCacheMaxUnackedRanges(true); + ManagedLedger ledger = factory.open("my_test_ledger" + UUID.randomUUID(), config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + long ledgerId = c1.getCursorLedgerHandle().getId(); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(4, 0, 4, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(5, 0, 5, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(6, 0, 6, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(7, 0, 7, 1); + CompletableFuture future = new CompletableFuture<>(); + c1.initialize(new PositionImpl(5, 5), Collections.emptyMap(), new ManagedCursorImpl.VoidCallback() { + @Override + public void operationComplete() { + future.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + future.completeExceptionally(exception); + } + }); + future.get(); + assertEquals(c1.getIndividuallyDeletedMessagesSet().size(), 4); + assertEquals(c1.getCursorLedgerHandle().getId(), ++ledgerId); + assertEquals(c1.getCursorLedgerHandle().getLastAddConfirmed(), 0); + + CompletableFuture future2 = new CompletableFuture<>(); + c1.pendingMarkDeleteOps.add(new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(5, 5), + new HashMap<>(), new AsyncCallbacks.MarkDeleteCallback() { + @Override + public void markDeleteComplete(Object ctx) { + future2.complete(null); + } + + @Override + public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { + future2.completeExceptionally(exception); + } + }, null)); + c1.internalFlushPendingMarkDeletes(); + future2.get(); + assertEquals(c1.getIndividuallyDeletedMessagesSet().toString(), "[(6:0..6:1],(7:0..7:1]]"); + assertEquals(c1.getCursorLedgerHandle().getId(), ledgerId); + assertEquals(c1.getRangeMarker().size(), 2); + assertTrue(c1.getRangeMarker().containsKey(6L)); + assertTrue(c1.getRangeMarker().containsKey(7L)); + + long entryId = c1.getRangeMarker().get(6L).getEntryId(); + long newLedgerId = ledgerId; + Enumeration entries = c1.getCursorLedgerHandle() + .readEntries(entryId, entryId + 2); + int counter = 0; + long startLedgerId = 6; + MLDataFormats.NestedPositionInfo.Builder positionBuilder = MLDataFormats.NestedPositionInfo.newBuilder(); + while (entries.hasMoreElements()) { + LedgerEntry entry = entries.nextElement(); + MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(entry.getEntryInputStream()); + if (counter == 2) { + //the last entry is marker + assertEquals(positionInfo.getMarkerIndexInfoCount(), 4); + // start from 4, entryId is 6 + AtomicLong subCounter = new AtomicLong(entryId - 2); + positionInfo.getMarkerIndexInfoList().forEach(markerIndexInfo -> { + assertEquals(markerIndexInfo.getEntryPosition().getLedgerId(), newLedgerId); + assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), subCounter.getAndIncrement()); + }); + continue; + } else { + assertEquals(positionInfo.getMarkerIndexInfoCount(), 0); + } + MLDataFormats.MessageRange range = positionInfo.getIndividualDeletedMessagesList().get(0); + assertEquals(range.getLowerEndpoint(), + positionBuilder.setLedgerId(startLedgerId + counter).setEntryId(0).build()); + assertEquals(range.getUpperEndpoint(), + positionBuilder.setLedgerId(startLedgerId + counter).setEntryId(1).build()); + counter++; + } + + c1.close(); + ledger.close(); + } + + @Test + public void testRecovery() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setEnableLruCacheMaxUnackedRanges(true); + String name = "my_test_ledger" + UUID.randomUUID(); + ManagedLedger ledger = factory.open(name, config); + ManagedCursorImpl cursor = initCursorAndData(ledger); + Optional optionalPositionInfo = + cursor.getLastAvailableMarker(cursor.getCursorLedgerHandle()); + assertTrue(optionalPositionInfo.isPresent()); + List markerIndexInfos = optionalPositionInfo.get().getMarkerIndexInfoList(); + assertEquals(markerIndexInfos.size(), 4); + + MetaStore mockMetaStore = mock(MetaStore.class); + doAnswer(new Answer() { + public Object answer(InvocationOnMock invocation) { + MLDataFormats.ManagedCursorInfo info = MLDataFormats.ManagedCursorInfo.newBuilder() + .setCursorsLedgerId(cursor.getCursorLedger()) + .setMarkDeleteLedgerId(3).setMarkDeleteEntryId(1) + .setLastActive(0L).build(); + Stat stat = mock(Stat.class); + MetaStore.MetaStoreCallback callback = + (MetaStore.MetaStoreCallback) invocation.getArguments()[2]; + callback.operationComplete(info, stat); + return null; + } + }).when(mockMetaStore).asyncGetCursorInfo(eq(name), eq("c1"), any(MetaStore.MetaStoreCallback.class)); + cursor.getRangeMarker().clear(); + cursor.getIndividuallyDeletedMessagesSet().clear(); + CompletableFuture future = new CompletableFuture<>(); + cursor.recover(new ManagedCursorImpl.VoidCallback() { + @Override + public void operationComplete() { + future.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + future.completeExceptionally(exception); + } + }); + future.get(); + // markDelete position is (4,5] + assertEquals(cursor.getRangeMarker().size(), 3); + assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 3); + } + + private ManagedCursorImpl initCursorAndData(ManagedLedger ledger) throws InterruptedException, ManagedLedgerException, java.util.concurrent.ExecutionException { + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + long ledgerId = c1.getCursorLedgerHandle().getId(); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(4, 0, 4, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(5, 0, 5, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(6, 0, 6, 1); + c1.getIndividuallyDeletedMessagesSet().addOpenClosed(7, 0, 7, 1); + CompletableFuture future = new CompletableFuture<>(); + c1.initialize(new PositionImpl(4, 5), Collections.emptyMap(), new ManagedCursorImpl.VoidCallback() { + @Override + public void operationComplete() { + future.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + future.completeExceptionally(exception); + } + }); + future.get(); + assertEquals(c1.getIndividuallyDeletedMessagesSet().size(), 4); + assertEquals(c1.getCursorLedgerHandle().getId(), ++ledgerId); + assertEquals(c1.getCursorLedgerHandle().getLastAddConfirmed(), 0); + + CompletableFuture future2 = new CompletableFuture<>(); + c1.pendingMarkDeleteOps.add(new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(4, 5), + new HashMap<>(), new AsyncCallbacks.MarkDeleteCallback() { + @Override + public void markDeleteComplete(Object ctx) { + future2.complete(null); + } + + @Override + public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { + future2.completeExceptionally(exception); + } + }, null)); + c1.internalFlushPendingMarkDeletes(); + future2.get(); + return c1; + } + + @Test + public void testCompatibility() throws Exception { + } } From 9e3137f144cee3ec25042fb9e6ace8c313ccdec5 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Sun, 23 May 2021 21:53:49 +0800 Subject: [PATCH 08/18] add unit test --- .../mledger/impl/ManagedCursorImpl.java | 53 +++++++------------ .../mledger/impl/RangeSetWrapper.java | 4 +- .../mledger/impl/MultiEntryPositionTest.java | 32 ++++++----- .../mledger/impl/RangeSetWrapperTest.java | 1 + .../client/PulsarMockLedgerHandle.java | 2 +- 5 files changed, 43 insertions(+), 49 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 8071c9e0e8a7f..f4ec52dc2972c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -364,12 +364,15 @@ void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback) { } PositionInfo markerPosition = optional.get(); try { - rangeMarker.clear(); + + Map tempMarker = new HashMap<>(); for (MLDataFormats.MarkerIndexInfo markerIndexInfo : markerPosition.getMarkerIndexInfoList()) { MLDataFormats.NestedPositionInfo nestedPositionInfo = markerIndexInfo.getEntryPosition(); - LedgerEntry ledgerEntry = readEntries(lh, nestedPositionInfo.getEntryId()); - if (ledgerEntry != null) { - PositionInfo entryPosition = PositionInfo.parseDelimitedFrom(ledgerEntry.getEntryInputStream()); + Enumeration entryEnumeration = lh.readEntries(nestedPositionInfo.getEntryId(), + nestedPositionInfo.getEntryId()); + if (entryEnumeration.hasMoreElements()) { + LedgerEntry ledgerEntry = entryEnumeration.nextElement(); + PositionInfo entryPosition = PositionInfo.parseFrom(ledgerEntry.getEntry()); if (entryPosition.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(entryPosition.getIndividualDeletedMessagesList(), false); } @@ -377,13 +380,16 @@ void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback) { && entryPosition.getBatchedEntryDeletionIndexInfoCount() > 0) { recoverBatchDeletedIndexes(entryPosition.getBatchedEntryDeletionIndexInfoList(), false); } - rangeMarker.put(markerIndexInfo.getTargetLedgerId(), nestedPositionInfo); + tempMarker.put(markerIndexInfo.getTargetLedgerId(), nestedPositionInfo); } if (individualDeletedMessages.size() > config.getMaxUnackedRangesInMemoryBytes()) { break; } } + rangeMarker.clear(); + rangeMarker.putAll(tempMarker); } catch (Exception e) { + log.error("recover from entry failed", e); initialize(getRollbackPosition(info), Collections.emptyMap(), callback); } recoveredCursor(new PositionImpl(markerPosition.getLedgerId(), markerPosition.getEntryId()), @@ -615,25 +621,6 @@ public void operationFailed(ManagedLedgerException exception) { }); } - public LedgerEntry readEntries(LedgerHandle ledgerHandle, long entryId) { - CompletableFuture future = new CompletableFuture<>(); - ledgerHandle.asyncReadEntries(entryId, entryId, (rc1, lh1, seq, ctx1) -> { - if (rc1 != BKException.Code.OK) { - future.completeExceptionally(BKException.create(rc1)); - } - if (seq.hasMoreElements()) { - future.complete(seq.nextElement()); - } else { - future.complete(null); - } - }, null); - try { - return future.get(config.getMetadataOperationsTimeoutSeconds(), TimeUnit.SECONDS); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - @Override public List readEntries(int numberOfEntriesToRead) throws InterruptedException, ManagedLedgerException { checkArgument(numberOfEntriesToRead > 0); @@ -2808,12 +2795,6 @@ private CompletableFuture checkIfNeedCreateNewLruLedgerAndSwitch(L return future; } - /** - * - * @param oldHandle - * @param newHandle - * @return - */ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle, LedgerHandle newHandle) { CompletableFuture result = new CompletableFuture<>(); // copy all entries in marker to new ledger @@ -2871,11 +2852,12 @@ Optional getLastAvailableMarker(LedgerHandle ledgerHandle) { long entryId = ledgerHandle.getLastAddConfirmed(); try { for (long i = entryId; i >= 0; i--) { - LedgerEntry entry = readEntries(ledgerHandle, i); - if (entry == null) { + Enumeration entryEnumeration = ledgerHandle.readEntries(i, i); + if (!entryEnumeration.hasMoreElements()) { return Optional.empty(); } - PositionInfo positionInfo = PositionInfo.parseFrom(entry.getEntryInputStream()); + LedgerEntry ledgerEntry = entryEnumeration.nextElement(); + PositionInfo positionInfo = PositionInfo.parseFrom(ledgerEntry.getEntry()); Map propertiesMap = buildPropertiesMap(positionInfo.getPropertiesList()); if (!propertiesMap.containsKey(LRU_ENTRY) && !propertiesMap.containsKey(LRU_MARKER)) { log.info("Currently not using lru mode"); @@ -3473,6 +3455,11 @@ public boolean checkAndUpdateReadPositionChanged() { return isReadPositionOnTail || isReadPositionChanged; } + @VisibleForTesting + protected ManagedLedgerConfig getConfig() { + return config; + } + @VisibleForTesting protected Map getRangeMarker() { return rangeMarker; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index aa544e105d417..988edd9d74958 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -59,7 +59,7 @@ public class RangeSetWrapper> implements LongPairRangeSe public RangeSetWrapper(LongPairConsumer rangeConverter, ManagedCursorImpl managedCursor) { checkNotNull(managedCursor); - this.config = managedCursor.getManagedLedger().getConfig(); + this.config = managedCursor.getConfig(); this.managedCursor = managedCursor; this.rangeConverter = rangeConverter; this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() @@ -142,7 +142,7 @@ private void tryLoadLruRangeFromLedger(long key) { } try { Enumeration entryEnumeration = managedCursor.getCursorLedgerHandle() - .readEntries(positionInfo.getLedgerId(), positionInfo.getEntryId()); + .readEntries(positionInfo.getEntryId(), positionInfo.getEntryId()); if(entryEnumeration.hasMoreElements()){ LedgerEntry ledgerEntry = entryEnumeration.nextElement(); MLDataFormats.PositionInfo entryPosition = diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java index bd090993cb4b2..44e6739b83c13 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java @@ -421,25 +421,25 @@ public void testRecovery() throws Exception { String name = "my_test_ledger" + UUID.randomUUID(); ManagedLedger ledger = factory.open(name, config); ManagedCursorImpl cursor = initCursorAndData(ledger); + cursor.startCreatingNewMetadataLedger(); + Awaitility.await().untilAsserted(() -> assertEquals(cursor.getState(), "Open")); Optional optionalPositionInfo = cursor.getLastAvailableMarker(cursor.getCursorLedgerHandle()); assertTrue(optionalPositionInfo.isPresent()); List markerIndexInfos = optionalPositionInfo.get().getMarkerIndexInfoList(); - assertEquals(markerIndexInfos.size(), 4); + assertEquals(markerIndexInfos.size(), 3); MetaStore mockMetaStore = mock(MetaStore.class); - doAnswer(new Answer() { - public Object answer(InvocationOnMock invocation) { - MLDataFormats.ManagedCursorInfo info = MLDataFormats.ManagedCursorInfo.newBuilder() - .setCursorsLedgerId(cursor.getCursorLedger()) - .setMarkDeleteLedgerId(3).setMarkDeleteEntryId(1) - .setLastActive(0L).build(); - Stat stat = mock(Stat.class); - MetaStore.MetaStoreCallback callback = - (MetaStore.MetaStoreCallback) invocation.getArguments()[2]; - callback.operationComplete(info, stat); - return null; - } + doAnswer((Answer) invocation -> { + MLDataFormats.ManagedCursorInfo info = MLDataFormats.ManagedCursorInfo.newBuilder() + .setCursorsLedgerId(cursor.getCursorLedger()) + .setMarkDeleteLedgerId(3).setMarkDeleteEntryId(1) + .setLastActive(0L).build(); + Stat stat = mock(Stat.class); + MetaStore.MetaStoreCallback callback = + (MetaStore.MetaStoreCallback) invocation.getArguments()[2]; + callback.operationComplete(info, stat); + return null; }).when(mockMetaStore).asyncGetCursorInfo(eq(name), eq("c1"), any(MetaStore.MetaStoreCallback.class)); cursor.getRangeMarker().clear(); cursor.getIndividuallyDeletedMessagesSet().clear(); @@ -459,6 +459,7 @@ public void operationFailed(ManagedLedgerException exception) { // markDelete position is (4,5] assertEquals(cursor.getRangeMarker().size(), 3); assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 3); + assertEquals(cursor.getIndividuallyDeletedMessagesSet().toString(), "[(5:0..5:1],(6:0..6:1],(7:0..7:1]]"); } private ManagedCursorImpl initCursorAndData(ManagedLedger ledger) throws InterruptedException, ManagedLedgerException, java.util.concurrent.ExecutionException { @@ -506,4 +507,9 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { @Test public void testCompatibility() throws Exception { } + + @Test + public void testLoadLruRangeFromLedger() throws Exception { + + } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java index 995ea73fc9ff3..af35d7db808dc 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapperTest.java @@ -56,6 +56,7 @@ public void setUp() { orderedScheduler = OrderedScheduler.newSchedulerBuilder() .numThreads(2).build(); doReturn(managedLedgerConfig).when(managedLedger).getConfig(); + doReturn(managedLedgerConfig).when(managedCursor).getConfig(); doReturn(orderedScheduler).when(managedLedger).getScheduledExecutor(); doReturn(managedLedger).when(managedCursor).getManagedLedger(); } diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java index 8a62e42e05a6f..1238270281947 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java @@ -133,7 +133,7 @@ public LedgerEntry nextElement() { } else { cb.readComplete(BKException.Code.OK, PulsarMockLedgerHandle.this, res, ctx); } - }, bk.executor); + }); } @Override From 4d3c6e3fb773399453a37761c8c430a7a47dcdbd Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Sun, 23 May 2021 23:47:59 +0800 Subject: [PATCH 09/18] add unit test --- .../mledger/impl/RangeSetWrapper.java | 2 +- .../mledger/impl/MultiEntryPositionTest.java | 53 +++++++++++++++---- 2 files changed, 44 insertions(+), 11 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index 988edd9d74958..a176b83dd1f7b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -146,7 +146,7 @@ private void tryLoadLruRangeFromLedger(long key) { if(entryEnumeration.hasMoreElements()){ LedgerEntry ledgerEntry = entryEnumeration.nextElement(); MLDataFormats.PositionInfo entryPosition = - MLDataFormats.PositionInfo.parseDelimitedFrom(ledgerEntry.getEntryInputStream()); + MLDataFormats.PositionInfo.parseFrom(ledgerEntry.getEntry()); managedCursor.recoverIndividualDeletedMessages(entryPosition.getIndividualDeletedMessagesList(), false); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java index 44e6739b83c13..3b276fff8440a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java @@ -29,7 +29,6 @@ import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.metadata.api.Stat; import org.awaitility.Awaitility; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.testng.annotations.Test; import java.util.Arrays; @@ -460,6 +459,49 @@ public void operationFailed(ManagedLedgerException exception) { assertEquals(cursor.getRangeMarker().size(), 3); assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 3); assertEquals(cursor.getIndividuallyDeletedMessagesSet().toString(), "[(5:0..5:1],(6:0..6:1],(7:0..7:1]]"); + + // clean up and test setMaxUnackedRangesInMemoryBytes + cursor.getRangeMarker().clear(); + cursor.getIndividuallyDeletedMessagesSet().clear(); + config.setMaxUnackedRangesInMemoryBytes(0); + + CompletableFuture future2 = new CompletableFuture<>(); + cursor.recover(new ManagedCursorImpl.VoidCallback() { + @Override + public void operationComplete() { + future2.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + future2.completeExceptionally(exception); + } + }); + future2.get(); + assertEquals(cursor.getRangeMarker().size(), 1); + assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 1); + + } + + @Test + public void testCompatibility() throws Exception { + } + + @Test + public void testLoadLruRangeFromLedger() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setEnableLruCacheMaxUnackedRanges(true); + String name = "my_test_ledger" + UUID.randomUUID(); + ManagedLedger ledger = factory.open(name, config); + ManagedCursorImpl cursor = initCursorAndData(ledger); + cursor.startCreatingNewMetadataLedger(); + Awaitility.await().untilAsserted(() -> assertEquals(cursor.getState(), "Open")); + + cursor.getIndividuallyDeletedMessagesSet().clear(); + assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 0); + // trigger load lru entry + assertTrue(cursor.getIndividuallyDeletedMessagesSet().contains(6, 1)); + assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 1); } private ManagedCursorImpl initCursorAndData(ManagedLedger ledger) throws InterruptedException, ManagedLedgerException, java.util.concurrent.ExecutionException { @@ -503,13 +545,4 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { future2.get(); return c1; } - - @Test - public void testCompatibility() throws Exception { - } - - @Test - public void testLoadLruRangeFromLedger() throws Exception { - - } } From ed6829aae114dd6db07f47416cb445bbf0adbb5a Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Mon, 24 May 2021 00:27:53 +0800 Subject: [PATCH 10/18] add unit test --- .../mledger/impl/ManagedCursorImpl.java | 9 ++++- .../mledger/impl/RangeSetWrapper.java | 7 ++-- .../mledger/impl/MultiEntryPositionTest.java | 38 +++++++++---------- 3 files changed, 28 insertions(+), 26 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f4ec52dc2972c..6e9d62e699692 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -2697,7 +2697,7 @@ private synchronized void persistPositionToMultiEntry(LedgerHandle lh, MarkDelet try { clonedMarker = new ConcurrentHashMap<>(rangeMarker); // get range info and group by ledger ID - rangeGroupByLedgerId = getRangeGroupByLedgerId(dirtyRange); + rangeGroupByLedgerId = getRangeGroupByLedgerId(dirtyRange, mdEntry); deletionIndexInfoGroupByLedgerId = getDeletionIndexInfosGroupByLedgerId(dirtyRange); } finally { lock.readLock().unlock(); @@ -2912,12 +2912,13 @@ Map> getDeletionIndexInf } } - Map> getRangeGroupByLedgerId(Set dirtyRange) { + Map> getRangeGroupByLedgerId(Set dirtyRange, MarkDeleteEntry mdEntry) { lock.readLock().lock(); try { if (individualDeletedMessages.isEmpty()) { return new HashMap<>(); } + PositionImpl position = mdEntry == null ? new PositionImpl(-1, -1) : mdEntry.newPosition; Map> ledgerIdToMessageRange = Maps.newHashMapWithExpectedSize(individualDeletedMessages.size()); MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo @@ -2928,6 +2929,10 @@ Map> getRangeGroupByLedgerId(Set di if (dirtyRange != null && !dirtyRange.contains(positionRange.upperEndpoint().ledgerId)) { return true; } + if (position.getLedgerId() >= positionRange.upperEndpoint().ledgerId + && position.getEntryId() >= positionRange.lowerEndpoint().entryId) { + return true; + } nestedPositionBuilder.setLedgerId(positionRange.lowerEndpoint().ledgerId) .setEntryId(positionRange.lowerEndpoint().entryId); messageRangeBuilder.setLowerEndpoint(nestedPositionBuilder.build()); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index a176b83dd1f7b..5db9b92a11b21 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -257,15 +257,14 @@ public void run() { if (firstRange == null) { return; } - // use Iterator to avoid ConcurrentModifyException + // remove invalid key Iterator iterator = lruCounter.getKeys().iterator(); while (iterator.hasNext()) { long ledgerId = iterator.next(); - if (firstRange.lowerEndpoint().compareTo(rangeConverter.apply(ledgerId, Integer.MAX_VALUE - 1)) > 0) { + if (firstRange.lowerEndpoint().compareTo( + rangeConverter.apply(ledgerId, Integer.MAX_VALUE - 1)) > 0) { iterator.remove(); log.info("LruTask remove invalid key {}", ledgerId); - } else { - break; } } if (isReachLruSwitchThreshold() && lruCounter.size() > 1) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java index 3b276fff8440a..2f6e5ed25d530 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java @@ -64,7 +64,7 @@ public void testGetRangeGroupByLedgerId() throws Exception { c1.getIndividuallyDeletedMessagesSet().addOpenClosed(1, 0, 1, 1); c1.getIndividuallyDeletedMessagesSet().addOpenClosed(2, 0, 2, 1); - Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(null); + Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(null, null); assertEquals(rangeGroupByLedgerId.size(), 3); AtomicLong count = new AtomicLong(0); @@ -79,7 +79,7 @@ public void testGetRangeGroupByLedgerId() throws Exception { Set filter = new HashSet<>(); filter.add(2L); - rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(filter); + rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(filter, null); assertEquals(rangeGroupByLedgerId.size(), 1); assertTrue(rangeGroupByLedgerId.containsKey(2L)); c1.close(); @@ -145,7 +145,7 @@ public void testCopyLruEntriesToNewLedger() throws Exception { c1.getIndividuallyDeletedMessagesSet().addOpenClosed(0, 0, 0, 1); c1.getIndividuallyDeletedMessagesSet().addOpenClosed(1, 0, 1, 1); c1.getIndividuallyDeletedMessagesSet().addOpenClosed(2, 0, 2, 1); - Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(null); + Map> rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(null, null); MLDataFormats.PositionInfo.Builder builder = MLDataFormats.PositionInfo.newBuilder().setLedgerId(1).setEntryId(0); Map rangeMarker = new HashMap<>(); @@ -161,11 +161,11 @@ public void testCopyLruEntriesToNewLedger() throws Exception { // init marker c1.getRangeMarker().putAll(rangeMarker); c1.setLastMarkDeleteEntry(new ManagedCursorImpl.MarkDeleteEntry( - new PositionImpl(0, 0), new HashMap<>(), null, null)); + new PositionImpl(-1, -1), new HashMap<>(), null, null)); // init PendingMarkDeletes c1.getIndividuallyDeletedMessagesSet().addOpenClosed(3, 0, 3, 1); CompletableFuture future = new CompletableFuture<>(); - c1.pendingMarkDeleteOps.add(new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(0, 0), + c1.pendingMarkDeleteOps.add(new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(-1, -1), new HashMap<>(), new AsyncCallbacks.MarkDeleteCallback() { @Override public void markDeleteComplete(Object ctx) { @@ -277,7 +277,7 @@ public void testPersistPositionToLedger() throws Exception { c1.getIndividuallyDeletedMessagesSet().addOpenClosed(1, 0, 1, 1); c1.getIndividuallyDeletedMessagesSet().addOpenClosed(2, 0, 2, 1); CompletableFuture future = new CompletableFuture<>(); - c1.internalMarkDelete(new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(1, 10), new HashMap<>(), + c1.internalMarkDelete(new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(1, 10), Collections.emptyMap(), new AsyncCallbacks.MarkDeleteCallback() { @Override public void markDeleteComplete(Object ctx) { @@ -295,32 +295,29 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { // markDelete position is (1, 10] , so only [(2:0..2:1]] is left assertEquals(marker.size(), 1); MLDataFormats.NestedPositionInfo positionInfo = marker.get((long) 2); - assertEquals(positionInfo.getLedgerId(), ledgerId); - assertEquals(positionInfo.getEntryId(), 3); assertEquals(c1.getIndividuallyDeletedMessages(), "[(2:0..2:1]]"); - Enumeration entries = c1.getCursorLedgerHandle().readEntries(positionInfo.getEntryId() - 2, + Enumeration entries = c1.getCursorLedgerHandle().readEntries(positionInfo.getEntryId(), positionInfo.getEntryId() + 1); int counter = 0; MLDataFormats.NestedPositionInfo.Builder positionBuilder = MLDataFormats.NestedPositionInfo.newBuilder(); while (entries.hasMoreElements()) { LedgerEntry entry = entries.nextElement(); MLDataFormats.PositionInfo position = MLDataFormats.PositionInfo.parseFrom(entry.getEntryInputStream()); - if (counter == 3) { + if (counter == 1) { // the last one is marker - assertEquals(position.getMarkerIndexInfoCount(), 3); - AtomicLong subCounter = new AtomicLong(positionInfo.getEntryId() - 2); + assertEquals(position.getMarkerIndexInfoCount(), 1); position.getMarkerIndexInfoList().forEach(markerIndexInfo -> { assertEquals(markerIndexInfo.getEntryPosition().getLedgerId(), ledgerId); - assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), subCounter.getAndIncrement()); + assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), positionInfo.getEntryId()); }); - assertEquals(subCounter.get(), 4); continue; } else { assertEquals(position.getMarkerIndexInfoCount(), 0); } MLDataFormats.MessageRange range = position.getIndividualDeletedMessagesList().get(0); - assertEquals(range.getLowerEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(0).build()); - assertEquals(range.getUpperEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(1).build()); + //(2:0 .. 2:1] + assertEquals(range.getLowerEndpoint(), positionBuilder.setLedgerId(2).setEntryId(0).build()); + assertEquals(range.getUpperEndpoint(), positionBuilder.setLedgerId(2).setEntryId(1).build()); counter++; } } @@ -388,11 +385,12 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { while (entries.hasMoreElements()) { LedgerEntry entry = entries.nextElement(); MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(entry.getEntryInputStream()); + //the last entry is marker if (counter == 2) { - //the last entry is marker - assertEquals(positionInfo.getMarkerIndexInfoCount(), 4); - // start from 4, entryId is 6 - AtomicLong subCounter = new AtomicLong(entryId - 2); + // mark deleted position is (5,5] , ranges 6 & 7 were left + assertEquals(positionInfo.getMarkerIndexInfoCount(), 2); + // start from 6 + AtomicLong subCounter = new AtomicLong(entryId); positionInfo.getMarkerIndexInfoList().forEach(markerIndexInfo -> { assertEquals(markerIndexInfo.getEntryPosition().getLedgerId(), newLedgerId); assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), subCounter.getAndIncrement()); From 0fa8d2b1f84d20229195d2f46d3175071550a051 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Mon, 24 May 2021 11:35:55 +0800 Subject: [PATCH 11/18] add unit test --- .../mledger/impl/ManagedCursorImpl.java | 16 +++++++++++----- .../mledger/impl/MultiEntryPositionTest.java | 17 +++++++++++++++-- 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 6e9d62e699692..6bbe3514ccb14 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -55,6 +55,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -2698,7 +2699,7 @@ private synchronized void persistPositionToMultiEntry(LedgerHandle lh, MarkDelet clonedMarker = new ConcurrentHashMap<>(rangeMarker); // get range info and group by ledger ID rangeGroupByLedgerId = getRangeGroupByLedgerId(dirtyRange, mdEntry); - deletionIndexInfoGroupByLedgerId = getDeletionIndexInfosGroupByLedgerId(dirtyRange); + deletionIndexInfoGroupByLedgerId = getDeletionIndexInfosGroupByLedgerId(dirtyRange, mdEntry); } finally { lock.readLock().unlock(); } @@ -2874,7 +2875,7 @@ Optional getLastAvailableMarker(LedgerHandle ledgerHandle) { } Map> getDeletionIndexInfosGroupByLedgerId( - Set dirtyRange) { + Set dirtyRange, MarkDeleteEntry mdEntry) { lock.readLock().lock(); try { if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes == null || batchDeletedIndexes.isEmpty()) { @@ -2886,7 +2887,11 @@ Map> getDeletionIndexInf .newBuilder(); Map> ledgerIdToIndexInfo = Maps.newHashMapWithExpectedSize(batchDeletedIndexes.size()); - Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + ConcurrentNavigableMap subMap = + mdEntry == null ? batchDeletedIndexes + : batchDeletedIndexes.subMap(mdEntry.newPosition, true, + batchDeletedIndexes.lastKey(), true); + Iterator> iterator = subMap.entrySet().iterator(); while (iterator.hasNext() && ledgerIdToIndexInfo.size() < config.getMaxBatchDeletedIndexToPersist()) { Map.Entry entry = iterator.next(); if (dirtyRange != null && !dirtyRange.contains(entry.getKey().getLedgerId())) { @@ -2929,8 +2934,9 @@ Map> getRangeGroupByLedgerId(Set di if (dirtyRange != null && !dirtyRange.contains(positionRange.upperEndpoint().ledgerId)) { return true; } - if (position.getLedgerId() >= positionRange.upperEndpoint().ledgerId - && position.getEntryId() >= positionRange.lowerEndpoint().entryId) { + if (position.getLedgerId() > positionRange.upperEndpoint().ledgerId + || (position.getLedgerId() == positionRange.lowerEndpoint().ledgerId + && position.getEntryId() > positionRange.lowerEndpoint().entryId)) { return true; } nestedPositionBuilder.setLedgerId(positionRange.lowerEndpoint().ledgerId) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java index 2f6e5ed25d530..2968e38492240 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java @@ -82,6 +82,12 @@ public void testGetRangeGroupByLedgerId() throws Exception { rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(filter, null); assertEquals(rangeGroupByLedgerId.size(), 1); assertTrue(rangeGroupByLedgerId.containsKey(2L)); + + rangeGroupByLedgerId = c1.getRangeGroupByLedgerId(null, + new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(1, 0), null, null, null)); + assertEquals(rangeGroupByLedgerId.size(), 2); + assertTrue(rangeGroupByLedgerId.containsKey(1L)); + assertTrue(rangeGroupByLedgerId.containsKey(2L)); c1.close(); ledger.close(); } @@ -101,7 +107,7 @@ public void testGetDeletionIndexInfosGroupByLedgerId() throws Exception { c1.getBatchDeletedIndexes().put(new PositionImpl(2,0), list.get(2)); Map> map = c1. - getDeletionIndexInfosGroupByLedgerId(null); + getDeletionIndexInfosGroupByLedgerId(null, null); assertEquals(map.size(), 3); AtomicLong count = new AtomicLong(0); @@ -114,10 +120,16 @@ public void testGetDeletionIndexInfosGroupByLedgerId() throws Exception { Set filter = new HashSet<>(); filter.add(2L); - map = c1.getDeletionIndexInfosGroupByLedgerId(filter); + map = c1.getDeletionIndexInfosGroupByLedgerId(filter, null); assertEquals(map.size(), 1); assertTrue(map.containsKey(2L)); + map = c1.getDeletionIndexInfosGroupByLedgerId(null, new ManagedCursorImpl.MarkDeleteEntry(new PositionImpl(1, 0) + , null, null, null)); + assertEquals(map.size(), 2); + assertTrue(map.containsKey(1L)); + assertTrue(map.containsKey(2L)); + c1.close(); ledger.close(); list.forEach(BitSetRecyclable::recycle); @@ -483,6 +495,7 @@ public void operationFailed(ManagedLedgerException exception) { @Test public void testCompatibility() throws Exception { + } @Test From fd54b19bf7e561152c43ed134113ea031c086650 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Mon, 24 May 2021 14:08:51 +0800 Subject: [PATCH 12/18] add unit test --- .../mledger/impl/MultiEntryPositionTest.java | 47 +++++++++++-------- 1 file changed, 27 insertions(+), 20 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java index 2968e38492240..f0f254c672fce 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java @@ -27,9 +27,7 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.metadata.api.Stat; import org.awaitility.Awaitility; -import org.mockito.stubbing.Answer; import org.testng.annotations.Test; import java.util.Arrays; import java.util.Collections; @@ -44,10 +42,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertTrue; @@ -437,19 +431,6 @@ public void testRecovery() throws Exception { assertTrue(optionalPositionInfo.isPresent()); List markerIndexInfos = optionalPositionInfo.get().getMarkerIndexInfoList(); assertEquals(markerIndexInfos.size(), 3); - - MetaStore mockMetaStore = mock(MetaStore.class); - doAnswer((Answer) invocation -> { - MLDataFormats.ManagedCursorInfo info = MLDataFormats.ManagedCursorInfo.newBuilder() - .setCursorsLedgerId(cursor.getCursorLedger()) - .setMarkDeleteLedgerId(3).setMarkDeleteEntryId(1) - .setLastActive(0L).build(); - Stat stat = mock(Stat.class); - MetaStore.MetaStoreCallback callback = - (MetaStore.MetaStoreCallback) invocation.getArguments()[2]; - callback.operationComplete(info, stat); - return null; - }).when(mockMetaStore).asyncGetCursorInfo(eq(name), eq("c1"), any(MetaStore.MetaStoreCallback.class)); cursor.getRangeMarker().clear(); cursor.getIndividuallyDeletedMessagesSet().clear(); CompletableFuture future = new CompletableFuture<>(); @@ -490,12 +471,38 @@ public void operationFailed(ManagedLedgerException exception) { future2.get(); assertEquals(cursor.getRangeMarker().size(), 1); assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 1); - } @Test public void testCompatibility() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setEnableLruCacheMaxUnackedRanges(true); + String name = "my_test_ledger" + UUID.randomUUID(); + ManagedLedger ledger = factory.open(name, config); + ManagedCursorImpl cursor = initCursorAndData(ledger); + cursor.startCreatingNewMetadataLedger(); + Awaitility.await().untilAsserted(() -> assertEquals(cursor.getState(), "Open")); + // disable lru + ledger.getConfig().setEnableLruCacheMaxUnackedRanges(false); + cursor.getConfig().setEnableLruCacheMaxUnackedRanges(false); + cursor.lastMarkDeleteEntry = null; + cursor.getRangeMarker().clear(); + cursor.getIndividuallyDeletedMessagesSet().clear(); + CompletableFuture future = new CompletableFuture<>(); + cursor.recover(new ManagedCursorImpl.VoidCallback() { + @Override + public void operationComplete() { + future.complete(null); + } + @Override + public void operationFailed(ManagedLedgerException exception) { + future.completeExceptionally(exception); + } + }); + future.get(); + assertEquals(cursor.lastMarkDeleteEntry.newPosition.ledgerId, cursor.getCursorLedgerHandle().getLastAddConfirmed()); + assertEquals(cursor.lastMarkDeleteEntry.newPosition.entryId, -1); } @Test From 4805d674eb339464d7974cf6d45b008f8a457223 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Tue, 25 May 2021 14:53:47 +0800 Subject: [PATCH 13/18] add unit test --- conf/broker.conf | 2 +- .../mledger/impl/ManagedCursorImpl.java | 5 +- .../pulsar/broker/ServiceConfiguration.java | 2 +- .../persistent/PersistentTopicTest.java | 92 +++++++++++++++++++ 4 files changed, 97 insertions(+), 4 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 73350e2b88c1b..92d004c6d42a4 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -920,7 +920,7 @@ managedLedgerMaxUnackedRangesToPersistInZooKeeper=1000 # If enabled, the maximum "acknowledgment holes" will not be limited. # Only part of hotspot data will keep in memory by using LRU. -enableLruCacheMaxUnackedRanges=true +enableLruCacheMaxUnackedRanges=false # Max Bytes of UnackedRanges can keep in memory. maxUnackedRangesInMemoryBytes=3145728 diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 6bbe3514ccb14..1d963031bc573 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -2921,7 +2921,7 @@ Map> getRangeGroupByLedgerId(Set di lock.readLock().lock(); try { if (individualDeletedMessages.isEmpty()) { - return new HashMap<>(); + return Collections.emptyMap(); } PositionImpl position = mdEntry == null ? new PositionImpl(-1, -1) : mdEntry.newPosition; Map> ledgerIdToMessageRange = @@ -2930,6 +2930,7 @@ Map> getRangeGroupByLedgerId(Set di .newBuilder(); MLDataFormats.MessageRange.Builder messageRangeBuilder = MLDataFormats.MessageRange.newBuilder(); AtomicInteger acksSerializedSize = new AtomicInteger(0); + AtomicInteger counter = new AtomicInteger(0); individualDeletedMessages.forEach((positionRange) -> { if (dirtyRange != null && !dirtyRange.contains(positionRange.upperEndpoint().ledgerId)) { return true; @@ -2950,7 +2951,7 @@ Map> getRangeGroupByLedgerId(Set di List list = ledgerIdToMessageRange .computeIfAbsent(messageRange.getUpperEndpoint().getLedgerId(), (le) -> new ArrayList<>()); list.add(messageRange); - return ledgerIdToMessageRange.size() <= config.getMaxUnackedRangesToPersist(); + return counter.incrementAndGet() <= config.getMaxUnackedRangesToPersist(); }); this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); return ledgerIdToMessageRange; 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 5c35a9663a6b5..5cac53c734a92 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 @@ -1522,7 +1522,7 @@ public class ServiceConfiguration implements PulsarConfiguration { doc = "If enabled, the maximum \"acknowledgment holes\" will not be limited. \n" + "Only part of hotspot data will keep in memory by using LRU." ) - private boolean enableLruCacheMaxUnackedRanges = true; + private boolean enableLruCacheMaxUnackedRanges = false; @FieldContext( category = CATEGORY_STORAGE_OFFLOADING, doc = "Max Bytes of UnackedRanges can keep in memory, only use for lru cache" diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index a89858ecc7a00..9b227d1336067 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -26,17 +26,28 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; import com.google.common.collect.Sets; +import lombok.Cleanup; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.RangeSetWrapper; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -65,6 +76,7 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { super.internalCleanup(); + resetConfig(); } /** @@ -99,6 +111,86 @@ public void testCleanFailedUnloadTopic() throws Exception { producer.close(); } + @Test + public void testLruEntryAck() throws Exception { + final String topicName = "persistent://prop/ns-abc/test" + UUID.randomUUID(); + final int msgNum = 50; + final String sub = "my-sub"; + + conf.setEnableLruCacheMaxUnackedRanges(true); + restartBroker(); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscriptionName("my-sub").subscribe(); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).enableBatching(false).topic(topicName).create(); + for (int i = 0; i < msgNum; i++) { + producer.send("msg" + i); + } + List> ackedMsg = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + Message msg = consumer.receive(1, TimeUnit.SECONDS); + if (i % 2 == 0) { + consumer.acknowledge(msg); + ackedMsg.add(msg); + } + } + consumer.close(); + + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); + PersistentSubscription subscription = topic.getSubscription(sub); + ManagedCursorImpl cursor = (ManagedCursorImpl) subscription.getCursor(); + assertTrue(cursor.getIndividuallyDeletedMessagesSet() instanceof RangeSetWrapper); + assertEquals(cursor.getIndividuallyDeletedMessagesSet().toString(), "[(3:1..3:2],(3:3..3:4],(3:5..3:6],(3:7..3:8]]"); + RangeSetWrapper setWrapper = + (RangeSetWrapper) cursor.getIndividuallyDeletedMessagesSet(); + Field field = ManagedCursorImpl.class.getDeclaredField("rangeMarker"); + field.setAccessible(true); + Map rangeMarker = + (Map) field.get(cursor); + //assertEquals(rangeMarker.size(), 1); + assertTrue(setWrapper.getDirtyKeyRecorder().contains(3L)); + + + conf.setEnableLruCacheMaxUnackedRanges(true); + // trigger persist + restartBroker(); + + consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName("my-sub").subscribe(); + topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); + subscription = topic.getSubscription(sub); + cursor = (ManagedCursorImpl) subscription.getCursor(); + assertTrue(cursor.getIndividuallyDeletedMessagesSet() instanceof RangeSetWrapper); + setWrapper = + (RangeSetWrapper) cursor.getIndividuallyDeletedMessagesSet(); + rangeMarker = + (Map) field.get(cursor); + //assertEquals(rangeMarker.size(), 1); + assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 4); + assertEquals(cursor.getIndividuallyDeletedMessagesSet().toString(), + "[(3:1..3:2],(3:3..3:4],(3:5..3:6],(3:7..3:8]]"); + assertFalse(setWrapper.getDirtyKeyRecorder().contains(3L)); + System.out.println("getIndividuallyDeletedMessagesSet2:" + rangeMarker); + + Set> restMessages = new HashSet<>(); + while (true) { + Message msg = consumer.receive(1, TimeUnit.SECONDS); + if (msg == null) { + break; + } + assertTrue(restMessages.add(msg)); + } + assertTrue(setWrapper.getDirtyKeyRecorder().contains(3L)); + + assertEquals(restMessages.size(), msgNum - ackedMsg.size()); + for (Message message : ackedMsg) { + assertFalse(restMessages.contains(message)); + } + consumer.close(); + } + /** * Test validates if topic's dispatcher is stuck then broker can doscover and unblock it. * From aab72473acd2d5ea98932df9d9b3dd21b3852ef8 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Wed, 26 May 2021 00:15:36 +0800 Subject: [PATCH 14/18] add unit test --- .../persistent/PersistentTopicTest.java | 108 +++++++++++++++--- 1 file changed, 90 insertions(+), 18 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index 9b227d1336067..53fc4cb13a1a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -112,17 +112,22 @@ public void testCleanFailedUnloadTopic() throws Exception { } @Test - public void testLruEntryAck() throws Exception { + public void testLruEntryAckRecoveryFormZk() throws Exception { final String topicName = "persistent://prop/ns-abc/test" + UUID.randomUUID(); final int msgNum = 50; final String sub = "my-sub"; + cleanup(); conf.setEnableLruCacheMaxUnackedRanges(true); - restartBroker(); + setup(); Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) .acknowledgmentGroupTime(0, TimeUnit.SECONDS) .subscriptionName("my-sub").subscribe(); + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); + PersistentSubscription subscription = topic.getSubscription(sub); + ManagedCursorImpl cursor = (ManagedCursorImpl) subscription.getCursor(); + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING).enableBatching(false).topic(topicName).create(); for (int i = 0; i < msgNum; i++) { @@ -136,43 +141,110 @@ public void testLruEntryAck() throws Exception { ackedMsg.add(msg); } } + + assertTrue(cursor.getIndividuallyDeletedMessagesSet() instanceof RangeSetWrapper); + Awaitility.await().untilAsserted(() -> + assertEquals(cursor.getIndividuallyDeletedMessagesSet().toString(), "[(3:1..3:2],(3:3..3:4],(3:5..3:6],(3:7..3:8]]")); + RangeSetWrapper setWrapper = + (RangeSetWrapper) cursor.getIndividuallyDeletedMessagesSet(); + assertTrue(setWrapper.getDirtyKeyRecorder().contains(3L)); + + consumer.close(); + conf.setEnableLruCacheMaxUnackedRanges(true); + restartBroker(); + + consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName(sub).subscribe(); + topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); + subscription = topic.getSubscription(sub); + ManagedCursorImpl cursor2 = (ManagedCursorImpl) subscription.getCursor(); + assertTrue(cursor2.getIndividuallyDeletedMessagesSet() instanceof RangeSetWrapper); + setWrapper = + (RangeSetWrapper) cursor2.getIndividuallyDeletedMessagesSet(); + assertEquals(cursor2.getIndividuallyDeletedMessagesSet().size(), 4); + assertEquals(cursor2.getIndividuallyDeletedMessagesSet().toString(), + "[(3:1..3:2],(3:3..3:4],(3:5..3:6],(3:7..3:8]]"); + + Set> restMessages = new HashSet<>(); + while (true) { + Message msg = consumer.receive(1, TimeUnit.SECONDS); + if (msg == null) { + break; + } + assertTrue(restMessages.add(msg)); + } + assertTrue(setWrapper.getDirtyKeyRecorder().contains(3L)); + + assertEquals(restMessages.size(), msgNum - ackedMsg.size()); + for (Message message : ackedMsg) { + assertFalse(restMessages.contains(message)); + } consumer.close(); + } + + @Test + public void testLruEntryAckFromLedger() throws Exception { + final String topicName = "persistent://prop/ns-abc/test" + UUID.randomUUID(); + final int msgNum = 50; + final String sub = "my-sub"; + + cleanup(); + conf.setEnableLruCacheMaxUnackedRanges(true); + conf.setManagedLedgerMaxUnackedRangesToPersistInZooKeeper(0); + setup(); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscriptionName(sub).subscribe(); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).enableBatching(false).topic(topicName).create(); + for (int i = 0; i < msgNum; i++) { + producer.send("msg" + i); + } PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); PersistentSubscription subscription = topic.getSubscription(sub); - ManagedCursorImpl cursor = (ManagedCursorImpl) subscription.getCursor(); + final ManagedCursorImpl cursor = (ManagedCursorImpl) subscription.getCursor(); + + List> ackedMsg = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + Message msg = consumer.receive(1, TimeUnit.SECONDS); + if (i % 2 == 0) { + consumer.acknowledge(msg); + ackedMsg.add(msg); + } + } + assertTrue(cursor.getIndividuallyDeletedMessagesSet() instanceof RangeSetWrapper); - assertEquals(cursor.getIndividuallyDeletedMessagesSet().toString(), "[(3:1..3:2],(3:3..3:4],(3:5..3:6],(3:7..3:8]]"); + Awaitility.await().untilAsserted(() -> + assertEquals(cursor.getIndividuallyDeletedMessagesSet().toString(), "[(3:1..3:2],(3:3..3:4],(3:5..3:6],(3:7..3:8]]")); RangeSetWrapper setWrapper = (RangeSetWrapper) cursor.getIndividuallyDeletedMessagesSet(); Field field = ManagedCursorImpl.class.getDeclaredField("rangeMarker"); field.setAccessible(true); - Map rangeMarker = - (Map) field.get(cursor); - //assertEquals(rangeMarker.size(), 1); assertTrue(setWrapper.getDirtyKeyRecorder().contains(3L)); + consumer.close(); conf.setEnableLruCacheMaxUnackedRanges(true); + conf.setManagedLedgerMaxUnackedRangesToPersistInZooKeeper(0); // trigger persist restartBroker(); consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) - .subscriptionName("my-sub").subscribe(); + .subscriptionName(sub).subscribe(); topic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); subscription = topic.getSubscription(sub); - cursor = (ManagedCursorImpl) subscription.getCursor(); - assertTrue(cursor.getIndividuallyDeletedMessagesSet() instanceof RangeSetWrapper); + ManagedCursorImpl cursor2 = (ManagedCursorImpl) subscription.getCursor(); + assertTrue(cursor2.getIndividuallyDeletedMessagesSet() instanceof RangeSetWrapper); setWrapper = - (RangeSetWrapper) cursor.getIndividuallyDeletedMessagesSet(); - rangeMarker = - (Map) field.get(cursor); - //assertEquals(rangeMarker.size(), 1); - assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 4); - assertEquals(cursor.getIndividuallyDeletedMessagesSet().toString(), + (RangeSetWrapper) cursor2.getIndividuallyDeletedMessagesSet(); + Map rangeMarker = + (Map) field.get(cursor2); + assertEquals(rangeMarker.size(), 1); + assertEquals(cursor2.getIndividuallyDeletedMessagesSet().size(), 4); + assertEquals(cursor2.getIndividuallyDeletedMessagesSet().toString(), "[(3:1..3:2],(3:3..3:4],(3:5..3:6],(3:7..3:8]]"); - assertFalse(setWrapper.getDirtyKeyRecorder().contains(3L)); - System.out.println("getIndividuallyDeletedMessagesSet2:" + rangeMarker); Set> restMessages = new HashSet<>(); while (true) { From 8a8a4512ce42e276b06a2873a9cba3276f1da079 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Wed, 26 May 2021 11:11:07 +0800 Subject: [PATCH 15/18] Fix NPE when filtering read entries --- .../mledger/impl/ManagedCursorImpl.java | 93 +++++++++++++------ 1 file changed, 67 insertions(+), 26 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 1d963031bc573..c62dde6840f3a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -481,6 +481,9 @@ private Map getRecoveredProperties(PositionInfo positionInfo) { recoveredProperties = Maps.newHashMap(); for (int i = 0; i < positionInfo.getPropertiesCount(); i++) { LongProperty property = positionInfo.getProperties(i); + if (LRU_MARKER.equals(property.getName()) || LRU_ENTRY.equals(property.getName())) { + continue; + } recoveredProperties.put(property.getName(), property.getValue()); } } @@ -556,7 +559,9 @@ private void recoverBatchDeletedIndexes (List { if (batchDeletedIndexInfo.getDeleteSetCount() > 0) { long[] array = new long[batchDeletedIndexInfo.getDeleteSetCount()]; @@ -2106,40 +2111,76 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { * @return a list of entries not containing deleted messages */ List filterReadEntries(List entries) { + Range entriesRange = Range.closed((PositionImpl) entries.get(0).getPosition(), + (PositionImpl) entries.get(entries.size() - 1).getPosition()); + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Filtering entries {} - alreadyDeleted: {}", ledger.getName(), name, entriesRange, + individualDeletedMessages); + } lock.readLock().lock(); try { - Range entriesRange = Range.closed((PositionImpl) entries.get(0).getPosition(), - (PositionImpl) entries.get(entries.size() - 1).getPosition()); - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Filtering entries {} - alreadyDeleted: {}", ledger.getName(), name, entriesRange, - individualDeletedMessages); + if (!shouldLoadLruRange(entriesRange)) { + if (individualDeletedMessages.isEmpty() || !entriesRange.isConnected(individualDeletedMessages.span())) { + // There are no individually deleted messages in this entry list, no need to perform filtering + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] No filtering needed for entries {}", ledger.getName(), name, entriesRange); + } + return entries; + } else { + // Remove from the entry list all the entries that were already marked for deletion + return doFilterEntries(entries); + } } + } finally { + lock.readLock().unlock(); + } + return LoadLruRangeAndFilterReadEntries(entries); + } - if (individualDeletedMessages.isEmpty() || !entriesRange.isConnected(individualDeletedMessages.span())) { - // There are no individually deleted messages in this entry list, no need to perform filtering + private ArrayList doFilterEntries(List entries) { + return Lists.newArrayList(Collections2.filter(entries, entry -> { + boolean includeEntry = !individualDeletedMessages.contains( + ((PositionImpl) entry.getPosition()).getLedgerId(), + ((PositionImpl) entry.getPosition()).getEntryId()); + if (!includeEntry) { if (log.isDebugEnabled()) { - log.debug("[{}] [{}] No filtering needed for entries {}", ledger.getName(), name, entriesRange); + log.debug("[{}] [{}] Filtering entry at {} - already deleted", ledger.getName(), name, + entry.getPosition()); } - return entries; - } else { - // Remove from the entry list all the entries that were already marked for deletion - return Lists.newArrayList(Collections2.filter(entries, entry -> { - boolean includeEntry = !individualDeletedMessages.contains( - ((PositionImpl) entry.getPosition()).getLedgerId(), - ((PositionImpl) entry.getPosition()).getEntryId()); - if (!includeEntry) { - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Filtering entry at {} - already deleted", ledger.getName(), name, - entry.getPosition()); - } - entry.release(); - } - return includeEntry; - })); + entry.release(); + } + return includeEntry; + })); + } + + private boolean shouldLoadLruRange(Range entriesRange) { + if (!config.isEnableLruCacheMaxUnackedRanges()) { + return false; + } + if (entriesRange.isConnected(individualDeletedMessages.span())) { + return false; + } + long lowerLedgerId = entriesRange.lowerEndpoint().getLedgerId(); + long upperLedgerId = entriesRange.upperEndpoint().getLedgerId(); + if (lowerLedgerId == upperLedgerId) { + return rangeMarker.containsKey(lowerLedgerId); + } + for (long i = lowerLedgerId; i < upperLedgerId; i++) { + if (rangeMarker.containsKey(lowerLedgerId)) { + return true; } + } + return false; + } + + private ArrayList LoadLruRangeAndFilterReadEntries(List entries) { + // After loading from ledger, ranges will be written to individualDeletedMessages, so we need write lock. + lock.writeLock().lock(); + try { + return doFilterEntries(entries); } finally { - lock.readLock().unlock(); + lock.writeLock().unlock(); } } From 4f7c97f934105c9aa12fcdc007ca82dea6120c84 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Wed, 26 May 2021 21:59:13 +0800 Subject: [PATCH 16/18] fix unit test --- .../mledger/impl/ManagedCursorImpl.java | 314 ++++++++++-------- .../mledger/impl/RangeSetWrapper.java | 20 +- .../mledger/impl/ManagedCursorTest.java | 6 +- .../mledger/impl/ManagedLedgerTest.java | 2 +- .../mledger/impl/MultiEntryPositionTest.java | 30 +- .../persistent/PersistentTopicTest.java | 1 - .../client/PulsarMockLedgerHandle.java | 2 +- 7 files changed, 213 insertions(+), 162 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index e13a452b7cfff..c36caba5eac97 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -45,7 +45,6 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; -import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -171,7 +170,7 @@ public class ManagedCursorImpl implements ManagedCursor { private volatile Stat cursorLedgerStat; // The key is the ledger we are acknowledging, value is the position that saves the ack information. - private Map rangeMarker = new ConcurrentHashMap<>(); + final private Map rangeMarker = new ConcurrentHashMap<>(); private static final LongPairConsumer positionRangeConverter = PositionImpl::new; private static final LongPairConsumer recyclePositionRangeConverter = (key, value) -> { @@ -315,7 +314,7 @@ public void operationComplete(ManagedCursorInfo info, Stat stat) { PositionImpl recoveredPosition = new PositionImpl(info.getMarkDeleteLedgerId(), info.getMarkDeleteEntryId()); if (info.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(info.getIndividualDeletedMessagesList(), true); + recoverIndividualDeletedMessages(info.getIndividualDeletedMessagesList()); } Map recoveredProperties = Collections.emptyMap(); @@ -358,47 +357,72 @@ void recoverFromMultiEntry(ManagedCursorInfo info, VoidCallback callback) { long ledgerId = info.getCursorsLedgerId(); OpenCallback openCallback = (rc, lh, ctx) -> { log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc); - if (shouldRecoverFromEldestEntry(info, callback, ledgerId, rc, lh)) { + if (tryRecoverFromEldestEntry(info, callback, ledgerId, rc, lh)) { return; } - Optional optional = getLastAvailableMarker(lh); - if (!optional.isPresent()) { + Optional markerPositionInfo; + try { + // get last available marker + markerPositionInfo = getLastAvailableMarker(lh, null, null) + .get(config.getMetadataOperationsTimeoutSeconds(), TimeUnit.SECONDS); + } catch (Exception e) { + log.error("fail to get last available marker", e); initialize(getRollbackPosition(info), Collections.emptyMap(), callback); return; } - PositionInfo markerPosition = optional.get(); - try { - - Map tempMarker = new HashMap<>(); - for (MLDataFormats.MarkerIndexInfo markerIndexInfo : markerPosition.getMarkerIndexInfoList()) { - MLDataFormats.NestedPositionInfo nestedPositionInfo = markerIndexInfo.getEntryPosition(); - Enumeration entryEnumeration = lh.readEntries(nestedPositionInfo.getEntryId(), - nestedPositionInfo.getEntryId()); - if (entryEnumeration.hasMoreElements()) { - LedgerEntry ledgerEntry = entryEnumeration.nextElement(); - PositionInfo entryPosition = PositionInfo.parseFrom(ledgerEntry.getEntry()); - if (entryPosition.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(entryPosition.getIndividualDeletedMessagesList(), false); - } - if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null - && entryPosition.getBatchedEntryDeletionIndexInfoCount() > 0) { - recoverBatchDeletedIndexes(entryPosition.getBatchedEntryDeletionIndexInfoList(), false); - } - tempMarker.put(markerIndexInfo.getTargetLedgerId(), nestedPositionInfo); - } - if (individualDeletedMessages.size() > config.getMaxUnackedRangesInMemoryBytes()) { - break; - } - } + if (!markerPositionInfo.isPresent()) { + initialize(getRollbackPosition(info), Collections.emptyMap(), callback); + return; + } + PositionInfo markerPosition = markerPositionInfo.get(); + Map tempMarker = new HashMap<>(); + List> futures = new ArrayList<>(); + // load multi entries + for (MLDataFormats.MarkerIndexInfo markerIndexInfo : markerPosition.getMarkerIndexInfoList()) { + CompletableFuture future = new CompletableFuture<>(); + futures.add(future); + MLDataFormats.NestedPositionInfo nestedPositionInfo = markerIndexInfo.getEntryPosition(); + lh.asyncReadEntries(nestedPositionInfo.getEntryId(), nestedPositionInfo.getEntryId(), + (i, ledgerHandle, enumeration, o) -> { + synchronized (rangeMarker) { + if (individualDeletedMessages.size() > config.getMaxUnackedRangesInMemoryBytes()) { + tempMarker.put(markerIndexInfo.getTargetLedgerId(), nestedPositionInfo); + future.complete(null); + return; + } + if (enumeration.hasMoreElements()) { + LedgerEntry ledgerEntry = enumeration.nextElement(); + PositionInfo entryPosition = null; + try { + entryPosition = PositionInfo.parseFrom(ledgerEntry.getEntry()); + } catch (InvalidProtocolBufferException e) { + future.completeExceptionally(e); + return; + } + if (entryPosition.getIndividualDeletedMessagesCount() > 0) { + recoverIndividualDeletedMessages(entryPosition.getIndividualDeletedMessagesList(), false); + } + if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null + && entryPosition.getBatchedEntryDeletionIndexInfoCount() > 0) { + recoverBatchDeletedIndexes(entryPosition.getBatchedEntryDeletionIndexInfoList(), false); + } + tempMarker.put(markerIndexInfo.getTargetLedgerId(), nestedPositionInfo); + future.complete(null); + } + } + }, null); + } + FutureUtil.waitForAll(futures).thenAccept((res) -> { rangeMarker.clear(); rangeMarker.putAll(tempMarker); - } catch (Exception e) { + recoveredCursor(new PositionImpl(markerPosition.getLedgerId(), markerPosition.getEntryId()), + getRecoveredProperties(markerPosition), lh); + callback.operationComplete(); + }).exceptionally(e -> { log.error("recover from entry failed", e); initialize(getRollbackPosition(info), Collections.emptyMap(), callback); - } - recoveredCursor(new PositionImpl(markerPosition.getLedgerId(), markerPosition.getEntryId()), - getRecoveredProperties(markerPosition), lh); - callback.operationComplete(); + return null; + }); }; try { bookkeeper.asyncOpenLedger(ledgerId, digestType, config.getPassword(), openCallback, null); @@ -420,7 +444,7 @@ private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callbac if (log.isInfoEnabled()) { log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc); } - boolean shouldRecoverFromEldestEntry = shouldRecoverFromEldestEntry(info, callback, ledgerId, rc, lh); + boolean shouldRecoverFromEldestEntry = tryRecoverFromEldestEntry(info, callback, ledgerId, rc, lh); if (shouldRecoverFromEldestEntry) { return; } @@ -434,7 +458,7 @@ private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callbac if (isBkErrorNotRecoverable(rc1)) { log.error("[{}] Error reading from metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc1)); - // Rewind to eldest entry available + // Rewind to oldest entry available initialize(getRollbackPosition(info), Collections.emptyMap(), callback); return; } else if (rc1 != BKException.Code.OK) { @@ -458,11 +482,11 @@ private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callbac PositionImpl position = new PositionImpl(positionInfo); if (positionInfo.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList(), true); + recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); } if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { - recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList(), true); + recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); } recoveredCursor(position, recoveredProperties, lh); callback.operationComplete(); @@ -477,49 +501,12 @@ private void recoverFromSingleEntry(ManagedCursorInfo info, VoidCallback callbac } } - private Map getRecoveredProperties(PositionInfo positionInfo) { - Map recoveredProperties = Collections.emptyMap(); - if (positionInfo.getPropertiesCount() > 0) { - // Recover properties map - recoveredProperties = Maps.newHashMap(); - for (int i = 0; i < positionInfo.getPropertiesCount(); i++) { - LongProperty property = positionInfo.getProperties(i); - if (LRU_MARKER.equals(property.getName()) || LRU_ENTRY.equals(property.getName())) { - continue; - } - recoveredProperties.put(property.getName(), property.getValue()); - } - } - return recoveredProperties; - } - - private boolean shouldRecoverFromEldestEntry(ManagedCursorInfo info, VoidCallback callback, long ledgerId, int rc, - LedgerHandle lh) { - if (isBkErrorNotRecoverable(rc)) { - log.error("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, - BKException.getMessage(rc)); - // Rewind to eldest entry available - initialize(getRollbackPosition(info), Collections.emptyMap(), callback); - return true; - } else if (rc != BKException.Code.OK) { - log.warn("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, - BKException.getMessage(rc)); - callback.operationFailed(new ManagedLedgerException(BKException.getMessage(rc))); - return true; - } - - if (lh.getLastAddConfirmed() < 0) { - log.warn("[{}] Error reading from metadata ledger {} for consumer {}: No entries in ledger", - ledger.getName(), ledgerId, name); - // Rewind to last cursor snapshot available - initialize(getRollbackPosition(info), Collections.emptyMap(), callback); - return true; - } - return false; + protected void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { + recoverIndividualDeletedMessages(individualDeletedMessagesList, true); } protected void recoverIndividualDeletedMessages(List individualDeletedMessagesList, - boolean cleanOldData) { + boolean cleanOldData) { lock.writeLock().lock(); try { if (cleanOldData) { @@ -558,6 +545,51 @@ protected void recoverIndividualDeletedMessages(List } } + private Map getRecoveredProperties(PositionInfo positionInfo) { + Map recoveredProperties = Collections.emptyMap(); + if (positionInfo.getPropertiesCount() > 0) { + // Recover properties map + recoveredProperties = Maps.newHashMap(); + for (int i = 0; i < positionInfo.getPropertiesCount(); i++) { + LongProperty property = positionInfo.getProperties(i); + if (LRU_MARKER.equals(property.getName()) || LRU_ENTRY.equals(property.getName())) { + continue; + } + recoveredProperties.put(property.getName(), property.getValue()); + } + } + return recoveredProperties; + } + + private boolean tryRecoverFromEldestEntry(ManagedCursorInfo info, VoidCallback callback, long ledgerId, int rc, + LedgerHandle lh) { + if (isBkErrorNotRecoverable(rc)) { + log.error("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, + BKException.getMessage(rc)); + // Rewind to eldest entry available + initialize(getRollbackPosition(info), Collections.emptyMap(), callback); + return true; + } else if (rc != BKException.Code.OK) { + log.warn("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, + BKException.getMessage(rc)); + callback.operationFailed(new ManagedLedgerException(BKException.getMessage(rc))); + return true; + } + + if (lh.getLastAddConfirmed() < 0) { + log.warn("[{}] Error reading from metadata ledger {} for consumer {}: No entries in ledger", + ledger.getName(), ledgerId, name); + // Rewind to last cursor snapshot available + initialize(getRollbackPosition(info), Collections.emptyMap(), callback); + return true; + } + return false; + } + + private void recoverBatchDeletedIndexes (List batchDeletedIndexInfoList) { + recoverBatchDeletedIndexes(batchDeletedIndexInfoList, true); + } + private void recoverBatchDeletedIndexes (List batchDeletedIndexInfoList , boolean cleanOldData) { lock.writeLock().lock(); @@ -617,7 +649,7 @@ void initialize(PositionImpl position, Map properties, final VoidC ledger.getName(), name, messagesConsumedCounter, markDeletePosition, readPosition); } - createNewMetadataLedgerAndSwitch(new VoidCallback() { + createNewMetadataLedger(new VoidCallback() { @Override public void operationComplete() { STATE_UPDATER.set(ManagedCursorImpl.this, State.Open); @@ -690,16 +722,6 @@ public void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadE ledger.asyncReadEntries(op); } - public void asyncReadEntry(PositionImpl readPosition, Object ctx, ReadEntriesCallback callback) { - if (isClosed()) { - callback.readEntriesFailed(new ManagedLedgerException("Cursor was already closed"), ctx); - return; - } - PENDING_READ_OPS_UPDATER.incrementAndGet(this); - OpReadEntry op = OpReadEntry.create(this, readPosition, 1, callback, ctx, readPosition); - ledger.asyncReadEntries(op); - } - @Override public Entry getNthEntry(int n, IndividualDeletedEntries deletedEntries) throws InterruptedException, ManagedLedgerException { @@ -2163,7 +2185,8 @@ List filterReadEntries(List entries) { lock.readLock().lock(); try { if (!shouldLoadLruRange(entriesRange)) { - if (individualDeletedMessages.isEmpty() || !entriesRange.isConnected(individualDeletedMessages.span())) { + if (individualDeletedMessages.isEmpty() || individualDeletedMessages.span() == null + || !entriesRange.isConnected(individualDeletedMessages.span())) { // There are no individually deleted messages in this entry list, no need to perform filtering if (log.isDebugEnabled()) { log.debug("[{}] [{}] No filtering needed for entries {}", ledger.getName(), name, entriesRange); @@ -2177,14 +2200,16 @@ List filterReadEntries(List entries) { } finally { lock.readLock().unlock(); } + // after reading from ledger, data will be written to individualDeletedMessages + // we need write lock at this time return LoadLruRangeAndFilterReadEntries(entries); } private ArrayList doFilterEntries(List entries) { return Lists.newArrayList(Collections2.filter(entries, entry -> { boolean includeEntry = !individualDeletedMessages.contains( - ((PositionImpl) entry.getPosition()).getLedgerId(), - ((PositionImpl) entry.getPosition()).getEntryId()); + entry.getLedgerId(), + entry.getEntryId()); if (!includeEntry) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Filtering entry at {} - already deleted", ledger.getName(), name, @@ -2197,11 +2222,12 @@ private ArrayList doFilterEntries(List entries) { })); } - private boolean shouldLoadLruRange(Range entriesRange) { + public boolean shouldLoadLruRange(Range entriesRange) { if (!config.isEnableLruCacheMaxUnackedRanges()) { return false; } - if (entriesRange.isConnected(individualDeletedMessages.span())) { + Range range = individualDeletedMessages.span(); + if (range != null && entriesRange.isConnected(individualDeletedMessages.span())) { return false; } long lowerLedgerId = entriesRange.lowerEndpoint().getLedgerId(); @@ -2209,7 +2235,7 @@ private boolean shouldLoadLruRange(Range entriesRange) { if (lowerLedgerId == upperLedgerId) { return rangeMarker.containsKey(lowerLedgerId); } - for (long i = lowerLedgerId; i < upperLedgerId; i++) { + for (long i = lowerLedgerId; i <= upperLedgerId; i++) { if (rangeMarker.containsKey(lowerLedgerId)) { return true; } @@ -2218,7 +2244,6 @@ private boolean shouldLoadLruRange(Range entriesRange) { } private ArrayList LoadLruRangeAndFilterReadEntries(List entries) { - // After loading from ledger, ranges will be written to individualDeletedMessages, so we need write lock. lock.writeLock().lock(); try { return doFilterEntries(entries); @@ -2516,12 +2541,12 @@ void startCreatingNewMetadataLedger() { // Check if we can immediately switch to a new metadata ledger if (PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER.get(this) == 0) { - createNewMetadataLedgerAndSwitch(); + createNewMetadataLedger(); } } - void createNewMetadataLedgerAndSwitch() { - createNewMetadataLedgerAndSwitch(new VoidCallback() { + void createNewMetadataLedger() { + createNewMetadataLedger(new VoidCallback() { @Override public void operationComplete() { // We now have a new ledger where we can write @@ -2564,7 +2589,7 @@ void internalFlushPendingMarkDeletes() { internalMarkDelete(lastEntry); } - void createNewMetadataLedgerAndSwitch(final VoidCallback callback) { + void createNewMetadataLedger(final VoidCallback callback) { ledger.mbean.startCursorLedgerCreateOp(); doCreateNewMetadataLedger().thenAccept(newLedgerHandle -> { ledger.mbean.endCursorLedgerCreateOp(); @@ -2892,25 +2917,27 @@ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle Map clonedMap = new ConcurrentHashMap<>(rangeMarker); try { for (Map.Entry longPositionEntry : clonedMap.entrySet()) { - MLDataFormats.NestedPositionInfo position = longPositionEntry.getValue(); - Enumeration entryEnumeration = - oldHandle.readEntries(position.getEntryId(), position.getEntryId()); - if (entryEnumeration == null || !entryEnumeration.hasMoreElements()) { - log.warn("can not find entry {}:{}", oldHandle.getLedgerMetadata().getLedgerId(), - position.getEntryId()); - continue; - } - final CompletableFuture future = new CompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); futures.add(future); - LedgerEntry entry = entryEnumeration.nextElement(); - newHandle.asyncAddEntry(entry.getEntry(), (rc, ledgerHandle, entryId, ctx) -> { - if (rc != BKException.Code.OK) { - future.completeExceptionally(BKException.create(rc)); - } else { - clonedMap.put(longPositionEntry.getKey(), - buildPositionInfo(newHandle.getLedgerMetadata().getLedgerId(), entryId)); - future.complete(null); - } + MLDataFormats.NestedPositionInfo position = longPositionEntry.getValue(); + oldHandle.asyncReadEntries(position.getEntryId() + , position.getEntryId(), (i, ledgerHandle, entryEnumeration, o) -> { + if (entryEnumeration == null || !entryEnumeration.hasMoreElements()) { + log.warn("can not find entry {}:{}", oldHandle.getLedgerMetadata().getLedgerId(), + position.getEntryId()); + future.complete(null); + return; + } + LedgerEntry entry = entryEnumeration.nextElement(); + newHandle.asyncAddEntry(entry.getEntry(), (rc, handle, entryId, ctx) -> { + if (rc != BKException.Code.OK) { + future.completeExceptionally(BKException.create(rc)); + } else { + clonedMap.put(longPositionEntry.getKey(), + buildPositionInfo(newHandle.getLedgerMetadata().getLedgerId(), entryId)); + future.complete(null); + } + }, null); }, null); } } catch (Exception e) { @@ -2924,8 +2951,7 @@ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle return; } PositionInfo.Builder writerBuilder = getPositionBuilder(lastMarkDeleteEntry) - .addProperties(createLruMarkerTag()) - .addAllMarkerIndexInfo(buildMarkerIndexMap(clonedMap)); + .addProperties(createLruMarkerTag()).addAllMarkerIndexInfo(buildMarkerIndexMap(clonedMap)); saveMarker(newHandle, writerBuilder.build().toByteArray()).whenComplete((re, ex) -> { if (ex != null) { result.completeExceptionally(ex); @@ -2938,29 +2964,41 @@ private CompletableFuture copyLruEntriesToNewLedger(LedgerHandle oldHandle return result; } - Optional getLastAvailableMarker(LedgerHandle ledgerHandle) { - long entryId = ledgerHandle.getLastAddConfirmed(); - try { - for (long i = entryId; i >= 0; i--) { - Enumeration entryEnumeration = ledgerHandle.readEntries(i, i); - if (!entryEnumeration.hasMoreElements()) { - return Optional.empty(); - } - LedgerEntry ledgerEntry = entryEnumeration.nextElement(); - PositionInfo positionInfo = PositionInfo.parseFrom(ledgerEntry.getEntry()); + CompletableFuture> getLastAvailableMarker(LedgerHandle ledgerHandle, + Long startEntryId, + CompletableFuture> callback) { + long entryId = startEntryId == null ? ledgerHandle.getLastAddConfirmed() : startEntryId; + CompletableFuture> future = callback == null ? new CompletableFuture<>() : callback; + + ledgerHandle.readAsync(entryId, entryId).thenAcceptAsync((ledgerEntries) -> { + Iterator iterator = ledgerEntries.iterator(); + if (!iterator.hasNext()) { + future.complete(Optional.empty()); + return; + } + try { + PositionInfo positionInfo = PositionInfo.parseFrom(iterator.next().getEntryBytes()); Map propertiesMap = buildPropertiesMap(positionInfo.getPropertiesList()); if (!propertiesMap.containsKey(LRU_ENTRY) && !propertiesMap.containsKey(LRU_MARKER)) { log.info("Currently not using lru mode"); - return Optional.empty(); + future.complete(Optional.empty()); + return; } if (propertiesMap.containsKey(LRU_MARKER)) { - return Optional.of(positionInfo); + future.complete(Optional.of(positionInfo)); + return; } + } catch (Exception e) { + future.completeExceptionally(e); } - } catch (Exception e) { - log.error("fail to get last available marker", e); - } - return Optional.empty(); + long nextEntryId = entryId - 1; + if (nextEntryId >= 0) { + getLastAvailableMarker(ledgerHandle, nextEntryId, future); + return; + } + future.complete(Optional.empty()); + }); + return future; } Map> getDeletionIndexInfosGroupByLedgerId( @@ -3208,7 +3246,7 @@ void decrementPendingMarkDeleteCount() { if (state == State.SwitchingLedger) { // A metadata ledger switch was pending and now we can do it since we don't have any more // outstanding mark-delete requests - createNewMetadataLedgerAndSwitch(); + createNewMetadataLedger(); } } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index 5db9b92a11b21..6a87307775bdc 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -110,7 +110,7 @@ public LruCache getLruCounter() { public boolean contains(long key, long value) { lruTouch(key); boolean isContain = rangeSet.contains(key, value); - if (!isContain && managedCursor.getRangeMarker().containsKey(key)) { + if (!isContain && shouldLoadLruRangeFromLedger(key, value)) { tryLoadLruRangeFromLedger(key); return rangeSet.contains(key, value); } @@ -121,7 +121,7 @@ public boolean contains(long key, long value) { public Range rangeContaining(long key, long value) { lruTouch(key); Range range = rangeSet.rangeContaining(key, value); - if(range == null && managedCursor.getRangeMarker().containsKey(key)){ + if(range == null && shouldLoadLruRangeFromLedger(key, value)){ tryLoadLruRangeFromLedger(key); return rangeSet.rangeContaining(key, value); } @@ -134,6 +134,22 @@ public void removeAtMost(long key, long value) { rangeSet.removeAtMost(key, value); } + public boolean shouldLoadLruRangeFromLedger(long key, long value) { + if (!config.isEnableLruCacheMaxUnackedRanges()) { + return false; + } + T range = rangeConverter.apply(key, value); + Range lower = rangeSet.firstRange(); + Range upper = rangeSet.lastRange(); + if (lower == null || upper == null) { + return managedCursor.getRangeMarker().containsKey(key); + } + if (range.compareTo(lower.lowerEndpoint()) >= 0 || range.compareTo(upper.upperEndpoint()) <= 0) { + return false; + } + return true; + } + private void tryLoadLruRangeFromLedger(long key) { if (config.isEnableLruCacheMaxUnackedRanges()) { MLDataFormats.NestedPositionInfo positionInfo = managedCursor.getRangeMarker().get(key); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 81edd65f4e6b9..03327ffe5ad9b 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -2732,7 +2732,7 @@ public void testOutOfOrderDeletePersistenceAfterCrash() throws Exception { /** *
-     * Verifies that {@link ManagedCursorImpl#createNewMetadataLedgerAndSwitch()} cleans up orphan ledgers if fails to switch new
+     * Verifies that {@link ManagedCursorImpl#createNewMetadataLedger()} cleans up orphan ledgers if fails to switch new
      * ledger
      * 
* @throws Exception @@ -2745,7 +2745,7 @@ public void testLeakFailedLedgerOfManageCursor() throws Exception { ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); CountDownLatch latch = new CountDownLatch(1); - c1.createNewMetadataLedgerAndSwitch(new VoidCallback() { + c1.createNewMetadataLedger(new VoidCallback() { @Override public void operationComplete() { latch.countDown(); @@ -2766,7 +2766,7 @@ public void operationFailed(ManagedLedgerException exception) { CountDownLatch latch2 = new CountDownLatch(1); // create ledger will create ledgerId = 6 long ledgerId = 6; - c1.createNewMetadataLedgerAndSwitch(new VoidCallback() { + c1.createNewMetadataLedger(new VoidCallback() { @Override public void operationComplete() { latch2.countDown(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index a97da4649b995..97a7d5fc26e59 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -2750,7 +2750,7 @@ private boolean updateCusorMetadataByCreatingMetadataLedger(MutableObject failed = new MutableObject<>(); failed.setValue(false); CountDownLatch createLedgerDoneLatch = new CountDownLatch(1); - cursor2.getValue().createNewMetadataLedgerAndSwitch(new VoidCallback() { + cursor2.getValue().createNewMetadataLedger(new VoidCallback() { @Override public void operationComplete() { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java index f0f254c672fce..1b4ff422e4857 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MultiEntryPositionTest.java @@ -136,7 +136,7 @@ public void testGetDeletionIndexInfosGroupByLedgerId() throws Exception { * persistPositionToLedger -> remove individualDeletedMessages * @throws Exception */ - @Test + @Test(timeOut = 30000) public void testCopyLruEntriesToNewLedger() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setEnableLruCacheMaxUnackedRanges(true); @@ -191,41 +191,39 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { assertEquals(c1.getRangeMarker().size(), 4); // After copying, flushPendingMarkDeletes will be triggered. // Copying will occupy entryId 0-3, and flushPendingMarkDeletes will occupy 4-7. - AtomicLong entryIdCounter = new AtomicLong(4); + Set entryIds = new HashSet<>(Arrays.asList(4L, 5L, 6L, 7L)); + Set ledgerIds = new HashSet<>(Arrays.asList(0L, 1L, 2L, 3L)); AtomicLong ledgerIdCounter = new AtomicLong(0); long newLedgerId = ledgerId + 1; c1.getRangeMarker().forEach((key, value) -> { - // in IndividuallyDeletedMessages ledgerId is from 0 to 3 - assertEquals(key.longValue(), ledgerIdCounter.getAndIncrement()); + assertTrue(entryIds.remove(value.getEntryId())); + assertTrue(ledgerIds.remove(key)); assertEquals(value.getLedgerId(), newLedgerId); - assertEquals(value.getEntryId(), entryIdCounter.getAndIncrement()); }); // Verify the copied entry and marker Enumeration entries = c1.getCursorLedgerHandle().readEntries(0, 3); int counter = 0; + int markerNum = 0; MLDataFormats.NestedPositionInfo.Builder positionBuilder = MLDataFormats.NestedPositionInfo.newBuilder(); while (entries.hasMoreElements()) { LedgerEntry entry = entries.nextElement(); MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(entry.getEntryInputStream()); - if (counter == 3) { - //the last entry is marker + if (positionInfo.getMarkerIndexInfoCount() > 0) { + markerNum++; assertEquals(positionInfo.getMarkerIndexInfoCount(), 3); - AtomicLong subCounter = new AtomicLong(0); positionInfo.getMarkerIndexInfoList().forEach(markerIndexInfo -> { assertEquals(markerIndexInfo.getEntryPosition().getLedgerId(), newLedgerId); - assertEquals(markerIndexInfo.getEntryPosition().getEntryId(), subCounter.getAndIncrement()); }); continue; - } else { - assertEquals(positionInfo.getMarkerIndexInfoCount(), 0); } MLDataFormats.MessageRange range = positionInfo.getIndividualDeletedMessagesList().get(0); - assertEquals(range.getLowerEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(0).build()); - assertEquals(range.getUpperEndpoint(), positionBuilder.setLedgerId(counter).setEntryId(1).build()); + assertEquals(range.getLowerEndpoint().getEntryId(), 0); + assertEquals(range.getUpperEndpoint().getEntryId(), 1); counter++; } assertEquals(counter, 3); + assertEquals(markerNum, 1); // Verify entries and marker created by flushPendingMarkDeletes entries = c1.getCursorLedgerHandle().readEntries(4, 7); @@ -417,7 +415,7 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { ledger.close(); } - @Test + @Test(timeOut = 30000) public void testRecovery() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setEnableLruCacheMaxUnackedRanges(true); @@ -427,7 +425,7 @@ public void testRecovery() throws Exception { cursor.startCreatingNewMetadataLedger(); Awaitility.await().untilAsserted(() -> assertEquals(cursor.getState(), "Open")); Optional optionalPositionInfo = - cursor.getLastAvailableMarker(cursor.getCursorLedgerHandle()); + cursor.getLastAvailableMarker(cursor.getCursorLedgerHandle(), null, null).get(); assertTrue(optionalPositionInfo.isPresent()); List markerIndexInfos = optionalPositionInfo.get().getMarkerIndexInfoList(); assertEquals(markerIndexInfos.size(), 3); @@ -469,7 +467,7 @@ public void operationFailed(ManagedLedgerException exception) { } }); future2.get(); - assertEquals(cursor.getRangeMarker().size(), 1); + assertEquals(cursor.getRangeMarker().size(), 3); assertEquals(cursor.getIndividuallyDeletedMessagesSet().size(), 1); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index a4fd4600f41a7..089b7da55a7ce 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service.persistent; -import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doNothing; diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java index 1238270281947..8a62e42e05a6f 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java @@ -133,7 +133,7 @@ public LedgerEntry nextElement() { } else { cb.readComplete(BKException.Code.OK, PulsarMockLedgerHandle.this, res, ctx); } - }); + }, bk.executor); } @Override From 8e5bc5b1a45481721c94e5e05965b9ead39871a8 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Thu, 27 May 2021 20:52:20 +0800 Subject: [PATCH 17/18] remove unused code --- .../pulsar/common/util/collections/LongPairRangeSet.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java index b79e3baa32245..92ae3a659201b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java @@ -174,14 +174,6 @@ public long getValue() { return this.value; } - public void setKey(long key) { - this.key = key; - } - - public void setValue(long value) { - this.value = value; - } - @Override public int compareTo(LongPair o) { return ComparisonChain.start().compare(key, o.getKey()).compare(value, o.getValue()).result(); From 6942cd8dd0748f74d30ee1523e2953e31e497568 Mon Sep 17 00:00:00 2001 From: feynmanlin Date: Thu, 27 May 2021 21:19:37 +0800 Subject: [PATCH 18/18] fix unit test --- .../org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index c36caba5eac97..cbb24c571de0b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -2629,6 +2629,7 @@ public void operationFailed(ManagedLedgerException exception) { }); }).exceptionally(e -> { ledger.mbean.endCursorLedgerCreateOp(); + callback.operationFailed(createManagedLedgerException(e)); return null; }); @@ -2667,6 +2668,7 @@ CompletableFuture doCreateNewMetadataLedger() { ledger.asyncCreateLedger(bookkeeper, config, digestType, (rc, lh, ctx) -> { if (ledger.checkAndCompleteLedgerOpTask(rc, lh, ctx)) { + future.complete(null); return; }