diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AbstractEntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AbstractEntryImpl.java new file mode 100644 index 0000000000000..1997f8d370352 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AbstractEntryImpl.java @@ -0,0 +1,178 @@ +/* + * 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 io.netty.buffer.ByteBuf; +import io.netty.util.Recycler; +import io.netty.util.ReferenceCounted; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.util.AbstractCASReferenceCounted; + +public abstract class AbstractEntryImpl> extends AbstractCASReferenceCounted + implements Entry, Comparable { + protected final Recycler.Handle recyclerHandle; + protected long timestamp; + protected long ledgerId; + protected long entryId; + private ByteBuf data; + private int length = -1; + private Position position; + private Runnable onDeallocate; + + public AbstractEntryImpl(Recycler.Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + public long getTimestamp() { + return timestamp; + } + + @Override + public ByteBuf getDataBuffer() { + return data; + } + + protected void setDataBuffer(ByteBuf data) { + this.data = data; + this.length = data.readableBytes(); + } + + @Override + public byte[] getData() { + ByteBuf data = getDataBuffer().duplicate(); + byte[] array = new byte[data.readableBytes()]; + data.getBytes(data.readerIndex(), array); + return array; + } + + // Only for test + + @Override + public byte[] getDataAndRelease() { + byte[] array = getData(); + release(); + return array; + } + + @Override + public int getLength() { + if (length == -1) { + throw new IllegalStateException("Entry has no length. Call setDataBuffer to set the data buffer first."); + } + return length; + } + + @Override + public Position getPosition() { + if (position == null) { + position = PositionFactory.create(ledgerId, entryId); + } + return position; + } + + @Override + public long getLedgerId() { + return ledgerId; + } + + @Override + public long getEntryId() { + return entryId; + } + + @Override + public int compareTo(T other) { + if (this.ledgerId != other.ledgerId) { + return this.ledgerId < other.ledgerId ? -1 : 1; + } + + if (this.entryId != other.entryId) { + return this.entryId < other.entryId ? -1 : 1; + } + + return 0; + } + + @Override + public ReferenceCounted touch(Object hint) { + return this; + } + + public void onDeallocate(Runnable r) { + if (this.onDeallocate == null) { + this.onDeallocate = r; + } else { + // this is not expected to happen + Runnable previous = this.onDeallocate; + this.onDeallocate = () -> { + try { + previous.run(); + } finally { + r.run(); + } + }; + } + } + + @Override + protected final void deallocate() { + beforeDeallocate(); + // This method is called whenever the ref-count of the EntryImpl reaches 0, so that now we can recycle it + if (onDeallocate != null) { + try { + onDeallocate.run(); + } finally { + onDeallocate = null; + } + } + data.release(); + data = null; + length = -1; + timestamp = -1; + ledgerId = -1; + entryId = -1; + position = null; + beforeRecycle(); + recyclerHandle.recycle(self()); + } + + /** + * This method is called just before the object is deallocated. + * Subclasses can override this method to run actions before the fields + * of the object are cleared and the object gets recycled. + */ + protected void beforeDeallocate() { + // No-op + } + + /** + * This method is called just before the object is recycled. Subclasses can override this methods to cleanup + * the object before it is returned to the pool. + */ + protected void beforeRecycle() { + // No-op + } + + @SuppressWarnings("unchecked") + protected T self() { + return (T) this; + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java index e0e2b859794b5..4d75097a98c13 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java @@ -22,18 +22,13 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.util.Recycler; -import io.netty.util.Recycler.Handle; -import io.netty.util.ReferenceCounted; import org.apache.bookkeeper.client.api.LedgerEntry; +import org.apache.bookkeeper.client.impl.LedgerEntryImpl; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.util.AbstractCASReferenceCounted; -import org.apache.bookkeeper.mledger.util.RangeCache; - -public final class EntryImpl extends AbstractCASReferenceCounted implements Entry, Comparable, - RangeCache.ValueWithKeyValidation { +import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; +public final class EntryImpl extends AbstractEntryImpl { private static final Recycler RECYCLER = new Recycler() { @Override protected EntryImpl newObject(Handle handle) { @@ -41,22 +36,33 @@ protected EntryImpl newObject(Handle handle) { } }; - private final Handle recyclerHandle; - private long timestamp; - private long ledgerId; - private long entryId; - private Position position; - ByteBuf data; - - private Runnable onDeallocate; + public static EntryImpl create(LedgerEntry ledgerEntry, ManagedLedgerInterceptor interceptor) { + ManagedLedgerInterceptor.PayloadProcessorHandle processorHandle = null; + if (interceptor != null) { + ByteBuf duplicateBuffer = ledgerEntry.getEntryBuffer().retainedDuplicate(); + processorHandle = interceptor + .processPayloadBeforeEntryCache(duplicateBuffer); + if (processorHandle != null) { + ledgerEntry = LedgerEntryImpl.create(ledgerEntry.getLedgerId(), ledgerEntry.getEntryId(), + ledgerEntry.getLength(), processorHandle.getProcessedPayload()); + } else { + duplicateBuffer.release(); + } + } + EntryImpl returnEntry = create(ledgerEntry); + if (processorHandle != null) { + processorHandle.release(); + ledgerEntry.close(); + } + return returnEntry; + } public static EntryImpl create(LedgerEntry ledgerEntry) { EntryImpl entry = RECYCLER.get(); entry.timestamp = System.nanoTime(); entry.ledgerId = ledgerEntry.getLedgerId(); entry.entryId = ledgerEntry.getEntryId(); - entry.data = ledgerEntry.getEntryBuffer(); - entry.data.retain(); + entry.setDataBuffer(ledgerEntry.getEntryBuffer().retain()); entry.setRefCnt(1); return entry; } @@ -67,7 +73,7 @@ public static EntryImpl create(long ledgerId, long entryId, byte[] data) { entry.timestamp = System.nanoTime(); entry.ledgerId = ledgerId; entry.entryId = entryId; - entry.data = Unpooled.wrappedBuffer(data); + entry.setDataBuffer(Unpooled.wrappedBuffer(data)); entry.setRefCnt(1); return entry; } @@ -77,8 +83,7 @@ public static EntryImpl create(long ledgerId, long entryId, ByteBuf data) { entry.timestamp = System.nanoTime(); entry.ledgerId = ledgerId; entry.entryId = entryId; - entry.data = data; - entry.data.retain(); + entry.setDataBuffer(data.retain()); entry.setRefCnt(1); return entry; } @@ -88,128 +93,22 @@ public static EntryImpl create(Position position, ByteBuf data) { entry.timestamp = System.nanoTime(); entry.ledgerId = position.getLedgerId(); entry.entryId = position.getEntryId(); - entry.data = data; - entry.data.retain(); + entry.setDataBuffer(data.retain()); entry.setRefCnt(1); return entry; } - public static EntryImpl create(EntryImpl other) { + public static EntryImpl create(Entry other) { EntryImpl entry = RECYCLER.get(); entry.timestamp = System.nanoTime(); - entry.ledgerId = other.ledgerId; - entry.entryId = other.entryId; - entry.data = other.data.retainedDuplicate(); + entry.ledgerId = other.getLedgerId(); + entry.entryId = other.getEntryId(); + entry.setDataBuffer(other.getDataBuffer().retainedDuplicate()); entry.setRefCnt(1); return entry; } private EntryImpl(Recycler.Handle recyclerHandle) { - this.recyclerHandle = recyclerHandle; - } - - public void onDeallocate(Runnable r) { - if (this.onDeallocate == null) { - this.onDeallocate = r; - } else { - // this is not expected to happen - Runnable previous = this.onDeallocate; - this.onDeallocate = () -> { - try { - previous.run(); - } finally { - r.run(); - } - }; - } - } - - public long getTimestamp() { - return timestamp; - } - - @Override - public ByteBuf getDataBuffer() { - return data; - } - - @Override - public byte[] getData() { - byte[] array = new byte[data.readableBytes()]; - data.getBytes(data.readerIndex(), array); - return array; - } - - // Only for test - @Override - public byte[] getDataAndRelease() { - byte[] array = getData(); - release(); - return array; - } - - @Override - public int getLength() { - return data.readableBytes(); - } - - @Override - public Position getPosition() { - if (position == null) { - position = PositionFactory.create(ledgerId, entryId); - } - return position; - } - - @Override - public long getLedgerId() { - return ledgerId; - } - - @Override - public long getEntryId() { - return entryId; - } - - @Override - public int compareTo(EntryImpl other) { - if (this.ledgerId != other.ledgerId) { - return this.ledgerId < other.ledgerId ? -1 : 1; - } - - if (this.entryId != other.entryId) { - return this.entryId < other.entryId ? -1 : 1; - } - - return 0; - } - - @Override - public ReferenceCounted touch(Object hint) { - return this; - } - - @Override - protected void deallocate() { - // This method is called whenever the ref-count of the EntryImpl reaches 0, so that now we can recycle it - if (onDeallocate != null) { - try { - onDeallocate.run(); - } finally { - onDeallocate = null; - } - } - data.release(); - data = null; - timestamp = -1; - ledgerId = -1; - entryId = -1; - position = null; - recyclerHandle.recycle(this); - } - - @Override - public boolean matchesKey(Position key) { - return key.compareTo(ledgerId, entryId) == 0; + super(recyclerHandle); } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 627e3225519e6..bd50c0576e375 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -21,6 +21,7 @@ import static org.apache.bookkeeper.mledger.ManagedLedgerException.getManagedLedgerException; import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.NULL_OFFLOAD_PROMISE; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicates; import com.google.common.collect.BoundType; import com.google.common.collect.Maps; @@ -118,6 +119,7 @@ public class ManagedLedgerFactoryImpl implements ManagedLedgerFactory { private final ManagedLedgerFactoryConfig config; @Getter protected final OrderedScheduler scheduledExecutor; + @Getter private final ScheduledExecutorService cacheEvictionExecutor; @Getter @@ -147,6 +149,7 @@ public class ManagedLedgerFactoryImpl implements ManagedLedgerFactory { */ @Getter private boolean metadataServiceAvailable; + private final ManagedLedgerConfig defaultManagedLedgerConfig; private static class PendingInitializeManagedLedger { @@ -170,7 +173,8 @@ public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, ClientConfi ManagedLedgerFactoryConfig config) throws Exception { this(metadataStore, new DefaultBkFactory(bkClientConfiguration), - true /* isBookkeeperManaged */, config, NullStatsLogger.INSTANCE, OpenTelemetry.noop()); + true /* isBookkeeperManaged */, config, NullStatsLogger.INSTANCE, OpenTelemetry.noop(), + new ManagedLedgerConfig()); } public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookKeeper bookKeeper) @@ -181,7 +185,15 @@ public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookKeeper public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookKeeper bookKeeper, ManagedLedgerFactoryConfig config) throws Exception { - this(metadataStore, (policyConfig) -> CompletableFuture.completedFuture(bookKeeper), config); + this(metadataStore, bookKeeper, config, new ManagedLedgerConfig()); + } + + public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookKeeper bookKeeper, + ManagedLedgerFactoryConfig config, ManagedLedgerConfig defaultManagedLedgerConfig) + throws Exception { + this(metadataStore, (policyConfig) -> CompletableFuture.completedFuture(bookKeeper), + false /* isBookkeeperManaged */, config, NullStatsLogger.INSTANCE, OpenTelemetry.noop(), + defaultManagedLedgerConfig); } public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, @@ -189,7 +201,7 @@ public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, ManagedLedgerFactoryConfig config) throws Exception { this(metadataStore, bookKeeperGroupFactory, false /* isBookkeeperManaged */, - config, NullStatsLogger.INSTANCE, OpenTelemetry.noop()); + config, NullStatsLogger.INSTANCE, OpenTelemetry.noop(), new ManagedLedgerConfig()); } public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, @@ -198,7 +210,7 @@ public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, OpenTelemetry openTelemetry) throws Exception { this(metadataStore, bookKeeperGroupFactory, false /* isBookkeeperManaged */, - config, statsLogger, openTelemetry); + config, statsLogger, openTelemetry, new ManagedLedgerConfig()); } private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, @@ -206,7 +218,9 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, boolean isBookkeeperManaged, ManagedLedgerFactoryConfig config, StatsLogger statsLogger, - OpenTelemetry openTelemetry) throws Exception { + OpenTelemetry openTelemetry, + ManagedLedgerConfig defaultManagedLedgerConfig) throws Exception { + this.defaultManagedLedgerConfig = defaultManagedLedgerConfig; MetadataCompressionConfig compressionConfigForManagedLedgerInfo = config.getCompressionConfigForManagedLedgerInfo(); MetadataCompressionConfig compressionConfigForManagedCursorInfo = @@ -303,17 +317,23 @@ private synchronized void refreshStats() { lastStatTimestamp = now; } - private synchronized void doCacheEviction() { + @VisibleForTesting + public synchronized void doCacheEviction() { long maxTimestamp = System.nanoTime() - cacheEvictionTimeThresholdNanos; + entryCacheManager.doCacheEviction(maxTimestamp); + } - ledgers.values().forEach(mlfuture -> { - if (mlfuture.isDone() && !mlfuture.isCompletedExceptionally()) { - ManagedLedgerImpl ml = mlfuture.getNow(null); - if (ml != null) { - ml.doCacheEviction(maxTimestamp); - } - } - }); + @VisibleForTesting + public void waitForPendingCacheEvictions() { + try { + cacheEvictionExecutor.submit(() -> { + // no-op + }).get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (ExecutionException e) { + throw new RuntimeException(e); + } } /** @@ -329,7 +349,7 @@ public Map getManagedLedgers() { @Override public ManagedLedger open(String name) throws InterruptedException, ManagedLedgerException { - return open(name, new ManagedLedgerConfig()); + return open(name, defaultManagedLedgerConfig); } @Override @@ -365,7 +385,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { @Override public void asyncOpen(String name, OpenLedgerCallback callback, Object ctx) { - asyncOpen(name, new ManagedLedgerConfig(), callback, null, ctx); + asyncOpen(name, defaultManagedLedgerConfig, callback, null, ctx); } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index cb7fb2658c401..f0534cc290d41 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -162,6 +162,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { protected final String name; private final Map ledgerMetadata; protected final BookKeeper.DigestType digestType; + private final AtomicReference cacheEvictionPosition = new AtomicReference<>(); protected ManagedLedgerConfig config; protected Map propertiesMap; @@ -2263,6 +2264,7 @@ protected void asyncReadEntry(ReadHandle ledger, Position position, ReadEntryCal protected void asyncReadEntry(ReadHandle ledger, long firstEntry, long lastEntry, OpReadEntry opReadEntry, Object ctx) { + boolean shouldCacheEntry = opReadEntry.cursor.isCacheReadEntry(); if (config.getReadEntryTimeoutSeconds() > 0) { // set readOpCount to uniquely validate if ReadEntryCallbackWrapper is already recycled long readOpCount = READ_OP_COUNT_UPDATER.incrementAndGet(this); @@ -2270,11 +2272,9 @@ protected void asyncReadEntry(ReadHandle ledger, long firstEntry, long lastEntry ReadEntryCallbackWrapper readCallback = ReadEntryCallbackWrapper.create(name, ledger.getId(), firstEntry, opReadEntry, readOpCount, createdTime, ctx); lastReadCallback = readCallback; - entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, opReadEntry.cursor.isCacheReadEntry(), - readCallback, readOpCount); + entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, shouldCacheEntry, readCallback, readOpCount); } else { - entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, opReadEntry.cursor.isCacheReadEntry(), opReadEntry, - ctx); + entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, shouldCacheEntry, opReadEntry, ctx); } } @@ -2433,25 +2433,35 @@ public boolean hasMoreEntries(Position position) { return result; } - void doCacheEviction(long maxTimestamp) { - if (entryCache.getSize() > 0) { - entryCache.invalidateEntriesBeforeTimestamp(maxTimestamp); - } - } - // slowest reader position is earliest mark delete position when cacheEvictionByMarkDeletedPosition=true // it is the earliest read position when cacheEvictionByMarkDeletedPosition=false private void invalidateEntriesUpToSlowestReaderPosition() { if (entryCache.getSize() <= 0) { return; } - if (!activeCursors.isEmpty()) { - Position evictionPos = activeCursors.getSlowestReaderPosition(); - if (evictionPos != null) { - entryCache.invalidateEntries(evictionPos); + Position slowestReaderPosition = activeCursors.getSlowestReaderPosition(); + Position evictionPos = slowestReaderPosition != null ? slowestReaderPosition : PositionFactory.LATEST; + Position currentEvictionPosition = cacheEvictionPosition.getAndUpdate(currentValue -> { + if (currentValue == null || currentValue == PositionFactory.LATEST + || currentValue.compareTo(evictionPos) < 0) { + return evictionPos; + } else { + return currentValue; } - } else { - entryCache.clear(); + }); + // when currentEvictionPosition is null, it means there is no eviction task in progress + if (currentEvictionPosition == null) { + // start a new eviction task that will invalidate entries up to slowest reader position when the task + // gets executed. The cacheEvictionPosition could get updates by other threads before the task gets + // executed. This minimizes the number of eviction tasks that get executed. + getFactory().getCacheEvictionExecutor().execute(() -> { + Position latestEvictionPosition = cacheEvictionPosition.getAndSet(null); + if (latestEvictionPosition == PositionFactory.LATEST) { + entryCache.clear(); + } else if (latestEvictionPosition != null) { + entryCache.invalidateEntries(latestEvictionPosition); + } + }); } } @@ -2969,7 +2979,9 @@ protected void doDeleteLedgers(List ledgersToDelete) { NUMBER_OF_ENTRIES_UPDATER.addAndGet(this, -ls.getEntries()); TOTAL_SIZE_UPDATER.addAndGet(this, -ls.getSize()); - entryCache.invalidateAllEntries(ls.getLedgerId()); + getFactory().getCacheEvictionExecutor().execute(() -> { + entryCache.invalidateAllEntries(ls.getLedgerId()); + }); } } @@ -3932,7 +3944,6 @@ private void deactivateCursorByName(String cursorName) { } } - public void removeWaitingCursor(ManagedCursor cursor) { this.waitingCursors.remove(cursor); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheEvictionPolicy.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/CachedEntry.java similarity index 65% rename from managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheEvictionPolicy.java rename to managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/CachedEntry.java index 41284825fa76f..a260141db0aca 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheEvictionPolicy.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/CachedEntry.java @@ -18,20 +18,13 @@ */ package org.apache.bookkeeper.mledger.impl.cache; -import java.util.List; +import io.netty.util.ReferenceCounted; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.Position; /** - * Cache eviction policy abstraction interface. - * + * Interface for cached entries in the {@link RangeCache}. */ -public interface EntryCacheEvictionPolicy { - /** - * Perform the cache eviction of at least sizeToFree bytes on the supplied list of caches. - * - * @param caches - * the list of caches to consider - * @param sizeToFree - * the minimum size in bytes to be freed - */ - void doEviction(List caches, long sizeToFree); +public interface CachedEntry extends Entry, ReferenceCounted { + boolean matchesKey(Position key); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/CachedEntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/CachedEntryImpl.java new file mode 100644 index 0000000000000..c5ebc56a192a8 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/CachedEntryImpl.java @@ -0,0 +1,52 @@ +/* + * 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.cache; + +import io.netty.buffer.ByteBuf; +import io.netty.util.Recycler; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.AbstractEntryImpl; + +final class CachedEntryImpl extends AbstractEntryImpl implements CachedEntry { + private static final Recycler RECYCLER = new Recycler() { + @Override + protected CachedEntryImpl newObject(Handle handle) { + return new CachedEntryImpl(handle); + } + }; + + public static CachedEntryImpl create(Position position, ByteBuf data) { + CachedEntryImpl entry = RECYCLER.get(); + entry.timestamp = System.nanoTime(); + entry.ledgerId = position.getLedgerId(); + entry.entryId = position.getEntryId(); + entry.setDataBuffer(data.retainedDuplicate()); + entry.setRefCnt(1); + return entry; + } + + private CachedEntryImpl(Recycler.Handle recyclerHandle) { + super(recyclerHandle); + } + + @Override + public boolean matchesKey(Position key) { + return key != null && entryId == key.getEntryId() && ledgerId == key.getLedgerId(); + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java index c2c5cd6bff43e..5722893983479 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java @@ -21,15 +21,14 @@ import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; +import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.EntryImpl; -import org.apache.commons.lang3.tuple.Pair; /** * Cache of entries used by a single ManagedLedger. An EntryCache is compared to other EntryCache instances using their * size (the memory that is occupied by each of them). */ -public interface EntryCache extends Comparable { +public interface EntryCache { /** * @return the name of the cache @@ -42,11 +41,10 @@ public interface EntryCache extends Comparable { *

If the overall limit have been reached, this will trigger the eviction of other entries, possibly from * other EntryCache instances * - * @param entry - * the entry to be cached + * @param entry the entry to be cached * @return whether the entry was inserted in cache */ - boolean insert(EntryImpl entry); + boolean insert(Entry entry); /** * Remove from cache all the entries related to a ledger up to lastPosition excluded. @@ -56,8 +54,6 @@ public interface EntryCache extends Comparable { */ void invalidateEntries(Position lastPosition); - void invalidateEntriesBeforeTimestamp(long timestamp); - /** * Remove from the cache all the entries belonging to a specific ledger. * @@ -71,15 +67,6 @@ public interface EntryCache extends Comparable { */ void clear(); - /** - * Force the cache to drop entries to free space. - * - * @param sizeToFree - * the total memory size to free - * @return a pair containing the number of entries evicted and their total size - */ - Pair evictEntries(long sizeToFree); - /** * Read entries from the cache or from bookkeeper. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDefaultEvictionPolicy.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDefaultEvictionPolicy.java deleted file mode 100644 index bc67aeced1d5a..0000000000000 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDefaultEvictionPolicy.java +++ /dev/null @@ -1,96 +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.cache; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Collections.reverseOrder; -import java.util.ArrayList; -import java.util.List; -import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Default eviction policy. - * - *

This policy consider only the bigger caches for doing eviction. - * - *

The PercentOfSizeToConsiderForEviction parameter should always be bigger than the cacheEvictionWatermak, - * otherwisethe eviction cycle will free less memory than what was required. - */ -public class EntryCacheDefaultEvictionPolicy implements EntryCacheEvictionPolicy { - - private static final double PercentOfSizeToConsiderForEviction = 0.5; - - @Override - public void doEviction(List caches, long sizeToFree) { - checkArgument(sizeToFree > 0); - checkArgument(!caches.isEmpty()); - - caches.sort(reverseOrder()); - - long totalSize = 0; - for (EntryCache cache : caches) { - totalSize += cache.getSize(); - } - - // This algorithm apply the eviction only the group of caches whose combined size reaches the - // PercentOfSizeToConsiderForEviction - List cachesToEvict = new ArrayList(); - long cachesToEvictTotalSize = 0; - long sizeToConsiderForEviction = (long) (totalSize * PercentOfSizeToConsiderForEviction); - log.debug("Need to gather at least {} from caches", sizeToConsiderForEviction); - - int cacheIdx = 0; - while (cachesToEvictTotalSize < sizeToConsiderForEviction) { - // This condition should always be true, considering that we cannot free more size that what we have in - // cache - checkArgument(cacheIdx < caches.size()); - - EntryCache entryCache = caches.get(cacheIdx++); - cachesToEvictTotalSize += entryCache.getSize(); - cachesToEvict.add(entryCache); - - log.debug("Added cache {} with size {}", entryCache.getName(), entryCache.getSize()); - } - - int evictedEntries = 0; - long evictedSize = 0; - - for (EntryCache entryCache : cachesToEvict) { - // To each entryCache chosen to for eviction, we'll ask to evict a proportional amount of data - long singleCacheSizeToFree = (long) (sizeToFree * (entryCache.getSize() / (double) cachesToEvictTotalSize)); - - if (singleCacheSizeToFree == 0) { - // If the size of this cache went to 0, it probably means that its entries has been removed from the - // cache since the time we've computed the ranking - continue; - } - - Pair evicted = entryCache.evictEntries(singleCacheSizeToFree); - evictedEntries += evicted.getLeft(); - evictedSize += evicted.getRight(); - } - - log.info("Completed cache eviction. Removed {} entries from {} caches. ({} Mb)", evictedEntries, - cachesToEvict.size(), evictedSize / RangeEntryCacheManagerImpl.MB); - } - - private static final Logger log = LoggerFactory.getLogger(EntryCacheDefaultEvictionPolicy.class); -} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java index 92541a7a72578..afb8091dcb625 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java @@ -31,7 +31,6 @@ import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; -import org.apache.commons.lang3.tuple.Pair; /** * Implementation of cache that always read from BookKeeper. @@ -51,7 +50,7 @@ public String getName() { } @Override - public boolean insert(EntryImpl entry) { + public boolean insert(Entry entry) { return false; } @@ -68,16 +67,7 @@ public void clear() { } @Override - public Pair evictEntries(long sizeToFree) { - return Pair.of(0, (long) 0); - } - - @Override - public void invalidateEntriesBeforeTimestamp(long timestamp) { - } - - @Override - public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean isSlowestReader, + public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry, final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry).thenAcceptAsync( ledgerEntries -> { @@ -86,7 +76,7 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boole try { for (LedgerEntry e : ledgerEntries) { // Insert the entries at the end of the list (they will be unsorted for now) - EntryImpl entry = RangeEntryCacheManagerImpl.create(e, interceptor); + EntryImpl entry = EntryImpl.create(e, interceptor); entries.add(entry); totalSize += entry.getLength(); } @@ -119,7 +109,7 @@ public void asyncReadEntry(ReadHandle lh, Position position, AsyncCallbacks.Read Iterator iterator = ledgerEntries.iterator(); if (iterator.hasNext()) { LedgerEntry ledgerEntry = iterator.next(); - EntryImpl returnEntry = RangeEntryCacheManagerImpl.create(ledgerEntry, interceptor); + EntryImpl returnEntry = EntryImpl.create(ledgerEntry, interceptor); ml.getMbean().recordReadEntriesOpsCacheMisses(1, returnEntry.getLength()); ml.getFactory().getMbean().recordCacheMiss(1, returnEntry.getLength()); @@ -139,10 +129,4 @@ public void asyncReadEntry(ReadHandle lh, Position position, AsyncCallbacks.Read public long getSize() { return 0; } - - @Override - public int compareTo(EntryCache other) { - return Long.compare(getSize(), other.getSize()); - } - } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheManager.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheManager.java index f68989039e51b..853cb1b87632c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheManager.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheManager.java @@ -36,4 +36,6 @@ public interface EntryCacheManager { void updateCacheEvictionWatermark(double cacheEvictionWatermark); double getCacheEvictionWatermark(); + + void doCacheEviction(long maxTimestamp); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java index 8f10b7d260f27..c3bd97e5a34f1 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java @@ -227,7 +227,7 @@ public PendingRead(PendingReadKey key, this.ledgerCache = ledgerCache; } - public synchronized void attach(CompletableFuture> handle) { + public synchronized void attach(CompletableFuture> handle) { if (state != PendingReadState.INITIALISED) { // this shouldn't ever happen. this is here to prevent misuse in future changes throw new IllegalStateException("Unexpected state " + state + " for PendingRead for key " + key); @@ -269,7 +269,7 @@ private synchronized List completeAndRemoveFromC // this method isn't synchronized since that could lead to deadlocks private void readEntriesComplete(List callbacks, - List entriesToReturn) { + List entriesToReturn) { if (callbacks.size() == 1) { ReadEntriesCallbackWithContext first = callbacks.get(0); if (first.startEntry == key.startEntry @@ -285,7 +285,7 @@ private void readEntriesComplete(List callbacks, callback.callback.readEntriesComplete( copyEntries(entriesToReturn, callback.startEntry, callback.endEntry), callback.ctx); } - for (EntryImpl entry : entriesToReturn) { + for (Entry entry : entriesToReturn) { entry.release(); } } @@ -299,9 +299,9 @@ private void readEntriesFailed(List callbacks, T } } - private static List keepEntries(List list, long startEntry, long endEntry) { + private static List keepEntries(List list, long startEntry, long endEntry) { List result = new ArrayList<>((int) (endEntry - startEntry + 1)); - for (EntryImpl entry : list) { + for (Entry entry : list) { long entryId = entry.getEntryId(); if (startEntry <= entryId && entryId <= endEntry) { result.add(entry); @@ -312,9 +312,9 @@ private static List keepEntries(List list, long startEntry, lo return result; } - private static List copyEntries(List entriesToReturn, long startEntry, long endEntry) { + private static List copyEntries(List entriesToReturn, long startEntry, long endEntry) { List result = new ArrayList<>((int) (endEntry - startEntry + 1)); - for (EntryImpl entry : entriesToReturn) { + for (Entry entry : entriesToReturn) { long entryId = entry.getEntryId(); if (startEntry <= entryId && entryId <= endEntry) { EntryImpl entryCopy = EntryImpl.create(entry); @@ -350,7 +350,8 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldC CompletableFuture> readFromLeftFuture = recursiveReadMissingEntriesAsync(lh, shouldCacheEntry, findBestCandidateOutcome.missingOnLeft); CompletableFuture> readFromRightFuture = - recursiveReadMissingEntriesAsync(lh, shouldCacheEntry, findBestCandidateOutcome.missingOnRight); + recursiveReadMissingEntriesAsync(lh, shouldCacheEntry, + findBestCandidateOutcome.missingOnRight); readFromLeftFuture .thenCombine(readFromMidFuture, (left, mid) -> { List result = new ArrayList<>(left); @@ -376,7 +377,7 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldC } if (createdByThisThread.get()) { - CompletableFuture> readResult = rangeEntryCache.readFromStorage(lh, firstEntry, + CompletableFuture> readResult = rangeEntryCache.readFromStorage(lh, firstEntry, lastEntry, shouldCacheEntry); pendingRead.attach(readResult); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCache.java new file mode 100644 index 0000000000000..e4df2f71c5818 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCache.java @@ -0,0 +1,287 @@ +/* + * 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.cache; + +import io.netty.util.IllegalReferenceCountException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentNavigableMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Position; +import org.apache.commons.lang3.tuple.Pair; + +/** + * Special type of cache where get() and delete() operations can be done over a range of keys. + * The implementation avoids locks and synchronization by relying on ConcurrentSkipListMap for storing the entries. + * Since there are no locks, it's necessary to ensure that a single entry in the cache is removed exactly once. + * Removing an entry multiple times could result in the entries of the cache being released multiple times, + * even while they are still in use. This is prevented by using a custom wrapper around the value to store in the map + * that ensures that the value is removed from the map only if the exact same instance is present in the map. + * There's also a check that ensures that the value matches the key. This is used to detect races without impacting + * consistency. + */ +@Slf4j +class RangeCache { + private final ConcurrentNavigableMap entries; + private final RangeCacheRemovalQueue removalQueue; + private AtomicLong size; // Total size of values stored in cache + + /** + * Construct a new RangeCache. + */ + public RangeCache(RangeCacheRemovalQueue removalQueue) { + this.removalQueue = removalQueue; + this.entries = new ConcurrentSkipListMap<>(); + this.size = new AtomicLong(0); + } + + /** + * Insert. + * + * @param key + * @param value ref counted value with at least 1 ref to pass on the cache + * @return whether the entry was inserted in the cache + */ + public boolean put(Position key, CachedEntry value) { + // retain value so that it's not released before we put it in the cache and calculate the weight + value.retain(); + try { + if (!value.matchesKey(key)) { + throw new IllegalArgumentException("Value '" + value + "' does not match key '" + key + "'"); + } + long entrySize = value.getLength(); + boolean added = RangeCacheEntryWrapper.withNewInstance(this, key, value, entrySize, newWrapper -> { + if (removalQueue.addEntry(newWrapper) && entries.putIfAbsent(key, newWrapper) == null) { + this.size.addAndGet(entrySize); + return true; + } else { + // recycle the new wrapper as it was not used + newWrapper.recycle(); + return false; + } + }); + return added; + } finally { + value.release(); + } + } + + public boolean exists(Position key) { + return key != null ? entries.containsKey(key) : true; + } + + /** + * Get the value associated with the key and increment the reference count of it. + * The caller is responsible for releasing the reference. + */ + public CachedEntry get(Position key) { + return getValueFromWrapper(key, entries.get(key)); + } + + private CachedEntry getValueFromWrapper(Position key, RangeCacheEntryWrapper valueWrapper) { + if (valueWrapper == null) { + return null; + } else { + CachedEntry value = valueWrapper.getValue(key); + return getRetainedValueMatchingKey(key, value); + } + } + + /** + * @apiNote the returned value must be released if it's not null + */ + private CachedEntry getValueMatchingEntry(Map.Entry entry) { + CachedEntry valueMatchingEntry = RangeCacheEntryWrapper.getValueMatchingMapEntry(entry); + return getRetainedValueMatchingKey(entry.getKey(), valueMatchingEntry); + } + + // validates that the value matches the key and that the value has not been recycled + // which are possible due to the lack of exclusive locks in the cache and the use of reference counted objects + /** + * @apiNote the returned value must be released if it's not null + */ + private CachedEntry getRetainedValueMatchingKey(Position key, CachedEntry value) { + if (value == null) { + // the wrapper has been recycled and contains another key + return null; + } + try { + value.retain(); + } catch (IllegalReferenceCountException e) { + // Value was already deallocated + return null; + } + // check that the value matches the key and that there's at least 2 references to it since + // the cache should be holding one reference and a new reference was just added in this method + if (value.refCnt() > 1 && value.matchesKey(key)) { + return value; + } else { + // Value or IdentityWrapper was recycled and already contains another value + // release the reference added in this method + value.release(); + return null; + } + } + + /** + * + * @param first + * the first key in the range + * @param last + * the last key in the range (inclusive) + * @return a collections of the value found in cache + */ + public Collection getRange(Position first, Position last) { + List values = new ArrayList(); + + // Return the values of the entries found in cache + for (Map.Entry entry : entries.subMap(first, true, last, true) + .entrySet()) { + CachedEntry value = getValueMatchingEntry(entry); + if (value != null) { + values.add(value); + } + } + + return values; + } + + /** + * + * @param first + * @param last + * @param lastInclusive + * @return an pair of ints, containing the number of removed entries and the total size + */ + public Pair removeRange(Position first, Position last, boolean lastInclusive) { + if (log.isDebugEnabled()) { + log.debug("Removing entries in range [{}, {}], lastInclusive: {}", first, last, lastInclusive); + } + RangeCacheRemovalCounters counters = RangeCacheRemovalCounters.create(); + Map subMap = entries.subMap(first, true, last, lastInclusive); + for (Map.Entry entry : subMap.entrySet()) { + removeEntryWithWriteLock(entry.getKey(), entry.getValue(), counters); + } + return handleRemovalResult(counters); + } + + boolean removeEntryWithWriteLock(Position expectedKey, RangeCacheEntryWrapper entryWrapper, + RangeCacheRemovalCounters counters) { + return entryWrapper.withWriteLock(e -> { + if (e.key == null || e.key != expectedKey) { + // entry has already been removed + return false; + } + return removeEntry(e.key, e.value, e, counters, false); + }); + } + + /** + * Remove the entry from the cache. This must be called within a function passed to + * {@link RangeCacheEntryWrapper#withWriteLock(Function)}. + * @param key the expected key of the entry + * @param value the expected value of the entry + * @param entryWrapper the entry wrapper instance + * @param counters the removal counters + * @return true if the entry was removed, false otherwise + */ + boolean removeEntry(Position key, CachedEntry value, RangeCacheEntryWrapper entryWrapper, + RangeCacheRemovalCounters counters, boolean updateSize) { + // always remove the entry from the map + entries.remove(key, entryWrapper); + if (value == null) { + // the wrapper has already been recycled and contains another key + return false; + } + try { + // add extra retain to avoid value being released while we are removing it + value.retain(); + } catch (IllegalReferenceCountException e) { + return false; + } + try { + if (!value.matchesKey(key)) { + return false; + } + long removedSize = entryWrapper.markRemoved(key, value); + if (removedSize > -1) { + counters.entryRemoved(removedSize); + if (updateSize) { + size.addAndGet(-removedSize); + } + if (value.refCnt() > 1) { + // remove the cache reference + value.release(); + } else { + log.info("Unexpected refCnt {} for key {}, removed entry without releasing the value", + value.refCnt(), key); + } + return true; + } else { + return false; + } + } finally { + // remove the extra retain + value.release(); + } + } + + private Pair handleRemovalResult(RangeCacheRemovalCounters counters) { + size.addAndGet(-counters.removedSize); + Pair result = Pair.of(counters.removedEntries, counters.removedSize); + counters.recycle(); + return result; + } + + /** + * Just for testing. Getting the number of entries is very expensive on the concurrent map + */ + protected long getNumberOfEntries() { + return entries.size(); + } + + public long getSize() { + return size.get(); + } + + /** + * Remove all the entries from the cache. + * + * @return size of removed entries + */ + public Pair clear() { + if (log.isDebugEnabled()) { + log.debug("Clearing the cache with {} entries and size {}", entries.size(), size.get()); + } + RangeCacheRemovalCounters counters = RangeCacheRemovalCounters.create(); + while (!Thread.currentThread().isInterrupted()) { + Map.Entry entry = entries.firstEntry(); + if (entry == null) { + break; + } + removeEntryWithWriteLock(entry.getKey(), entry.getValue(), counters); + } + return handleRemovalResult(counters); + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheEntryWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheEntryWrapper.java new file mode 100644 index 0000000000000..95a231fd958a4 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheEntryWrapper.java @@ -0,0 +1,158 @@ +/* + * 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.cache; + +import io.netty.util.Recycler; +import java.util.Map; +import java.util.concurrent.locks.StampedLock; +import java.util.function.Function; +import org.apache.bookkeeper.mledger.Position; + +/** + * Wrapper around the value to store in Map. This is needed to ensure that a specific instance can be removed from + * the map by calling the {@link Map#remove(Object, Object)} method. Certain race conditions could result in the + * wrong value being removed from the map. The instances of this class are recycled to avoid creating new objects. + */ +class RangeCacheEntryWrapper { + private final Recycler.Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { + @Override + protected RangeCacheEntryWrapper newObject(Handle recyclerHandle) { + return new RangeCacheEntryWrapper(recyclerHandle); + } + }; + private final StampedLock lock = new StampedLock(); + Position key; + CachedEntry value; + RangeCache rangeCache; + long size; + long timestampNanos; + + private RangeCacheEntryWrapper(Recycler.Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + static R withNewInstance(RangeCache rangeCache, Position key, CachedEntry value, long size, + Function function) { + RangeCacheEntryWrapper entryWrapper = RECYCLER.get(); + StampedLock lock = entryWrapper.lock; + long stamp = lock.writeLock(); + try { + entryWrapper.rangeCache = rangeCache; + entryWrapper.key = key; + entryWrapper.value = value; + entryWrapper.size = size; + entryWrapper.timestampNanos = System.nanoTime(); + return function.apply(entryWrapper); + } finally { + lock.unlockWrite(stamp); + } + } + + /** + * Get the value associated with the key. Returns null if the key does not match the key. + * + * @param key the key to match + * @return the value associated with the key, or null if the value has already been recycled or the key does not + * match + */ + CachedEntry getValue(Position key) { + return getValueInternal(key, false); + } + + /** + * Get the value associated with the Map.Entry's key and value. Exact instance of the key is required to match. + * + * @param entry the entry which contains the key and {@link RangeCacheEntryWrapper} value to get the value from + * @return the value associated with the key, or null if the value has already been recycled or the key does not + * exactly match the same instance + */ + static CachedEntry getValueMatchingMapEntry(Map.Entry entry) { + return entry.getValue().getValueInternal(entry.getKey(), true); + } + + /** + * Get the value associated with the key. Returns null if the key does not match the key associated with the + * value. + * + * @param key the key to match + * @param requireSameKeyInstance when true, the matching will be restricted to exactly the same instance of the + * key as the one stored in the wrapper. This is used to avoid any races + * when retrieving or removing the entries from the cache when the key and value + * instances are available. + * @return the value associated with the key, or null if the key does not match + */ + private CachedEntry getValueInternal(Position key, boolean requireSameKeyInstance) { + long stamp = lock.tryOptimisticRead(); + Position localKey = this.key; + CachedEntry localValue = this.value; + if (!lock.validate(stamp)) { + stamp = lock.readLock(); + localKey = this.key; + localValue = this.value; + lock.unlockRead(stamp); + } + // check that the given key matches the key associated with the value in the entry + // this is used to detect if the entry has already been recycled and contains another key + // when requireSameKeyInstance is true, the key must be exactly the same instance as the one stored in the + // entry to match + if (localKey != key && (requireSameKeyInstance || localKey == null || !localKey.equals(key))) { + return null; + } + return localValue; + } + + /** + * Marks the entry as removed if the key and value match the current key and value. + * This method should only be called while holding the write lock within {@link #withWriteLock(Function)}. + * @param key the expected key of the entry + * @param value the expected value of the entry + * @return the size of the entry if the entry was removed, -1 otherwise + */ + long markRemoved(Position key, CachedEntry value) { + if (this.key != key || this.value != value) { + return -1; + } + rangeCache = null; + this.key = null; + this.value = null; + long removedSize = size; + size = 0; + timestampNanos = 0; + return removedSize; + } + + R withWriteLock(Function function) { + long stamp = lock.writeLock(); + try { + return function.apply(this); + } finally { + lock.unlockWrite(stamp); + } + } + + void recycle() { + rangeCache = null; + key = null; + value = null; + size = 0; + timestampNanos = 0; + recyclerHandle.recycle(this); + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheRemovalCounters.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheRemovalCounters.java new file mode 100644 index 0000000000000..bbbdb9dc619df --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheRemovalCounters.java @@ -0,0 +1,59 @@ +/* + * 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.cache; + +import io.netty.util.Recycler; + +/** + * Mutable object to store the number of entries and the total size removed from the cache. The instances + * are recycled to avoid creating new instances. + */ +class RangeCacheRemovalCounters { + private final Recycler.Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { + @Override + protected RangeCacheRemovalCounters newObject(Handle recyclerHandle) { + return new RangeCacheRemovalCounters(recyclerHandle); + } + }; + int removedEntries; + long removedSize; + + private RangeCacheRemovalCounters(Recycler.Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + static RangeCacheRemovalCounters create() { + RangeCacheRemovalCounters results = RECYCLER.get(); + results.removedEntries = 0; + results.removedSize = 0; + return results; + } + + void recycle() { + removedEntries = 0; + removedSize = 0; + recyclerHandle.recycle(this); + } + + public void entryRemoved(long size) { + removedSize += size; + removedEntries++; + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheRemovalQueue.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheRemovalQueue.java new file mode 100644 index 0000000000000..31c7e87b21e10 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheRemovalQueue.java @@ -0,0 +1,134 @@ +/* + * 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.cache; + +import static com.google.common.base.Preconditions.checkArgument; +import org.apache.commons.lang3.tuple.Pair; +import org.jctools.queues.MpscUnboundedArrayQueue; + +/** + * A central queue to hold entries that are scheduled for removal from all range cache instances. + * Removal of entries is done in a single thread to avoid contention. + * This queue is used to evict entries based on timestamp or based on size to free up space in the cache. + */ +class RangeCacheRemovalQueue { + // The removal queue is unbounded, but we allocate memory in chunks to avoid frequent memory allocations. + private static final int REMOVAL_QUEUE_CHUNK_SIZE = 128 * 1024; + private final MpscUnboundedArrayQueue removalQueue = new MpscUnboundedArrayQueue<>( + REMOVAL_QUEUE_CHUNK_SIZE); + + public Pair evictLEntriesBeforeTimestamp(long timestampNanos) { + return evictEntries( + (e, c) -> e.timestampNanos < timestampNanos ? EvictionResult.REMOVE : EvictionResult.STOP); + } + + public Pair evictLeastAccessedEntries(long sizeToFree) { + checkArgument(sizeToFree > 0); + return evictEntries( + (e, c) -> { + // stop eviction if we have already removed enough entries + if (c.removedSize >= sizeToFree) { + return EvictionResult.STOP; + } + return EvictionResult.REMOVE; + }); + } + + public boolean addEntry(RangeCacheEntryWrapper newWrapper) { + return removalQueue.offer(newWrapper); + } + + enum EvictionResult { + REMOVE, MISSING, STOP; + + boolean isContinueEviction() { + return this != STOP; + } + + boolean shouldRemoveFromQueue() { + return this == REMOVE || this == MISSING; + } + } + + interface EvictionPredicate { + EvictionResult test(RangeCacheEntryWrapper entry, RangeCacheRemovalCounters counters); + } + + /** + * Evict entries from the removal queue based on the provided eviction predicate. + * This method is synchronized to prevent multiple threads from removing entries simultaneously. + * An MPSC (Multiple Producer Single Consumer) queue is used as the removal queue, which expects a single consumer. + * + * @param evictionPredicate the predicate to determine if an entry should be evicted + * @return the number of entries and the total size removed from the cache + */ + private synchronized Pair evictEntries(EvictionPredicate evictionPredicate) { + RangeCacheRemovalCounters counters = RangeCacheRemovalCounters.create(); + handleQueue(evictionPredicate, counters); + return handleRemovalResult(counters); + } + + private void handleQueue(EvictionPredicate evictionPredicate, + RangeCacheRemovalCounters counters) { + while (!Thread.currentThread().isInterrupted()) { + RangeCacheEntryWrapper entry = removalQueue.peek(); + if (entry == null) { + break; + } + EvictionResult evictionResult = handleEviction(evictionPredicate, entry, counters); + if (evictionResult.shouldRemoveFromQueue()) { + // remove the peeked entry from the queue + removalQueue.poll(); + // recycle the entry after it has been removed from the queue + entry.recycle(); + } + if (!evictionResult.isContinueEviction()) { + break; + } + } + } + + private EvictionResult handleEviction(EvictionPredicate evictionPredicate, RangeCacheEntryWrapper entry, + RangeCacheRemovalCounters counters) { + EvictionResult evictionResult = entry.withWriteLock(e -> { + EvictionResult result = + evaluateEvictionPredicate(evictionPredicate, counters, e); + if (result == EvictionResult.REMOVE) { + e.rangeCache.removeEntry(e.key, e.value, e, counters, true); + } + return result; + }); + return evictionResult; + } + + private static EvictionResult evaluateEvictionPredicate(EvictionPredicate evictionPredicate, + RangeCacheRemovalCounters counters, RangeCacheEntryWrapper entry) { + if (entry.key == null) { + // entry has been removed by another thread + return EvictionResult.MISSING; + } + return evictionPredicate.test(entry, counters); + } + + private Pair handleRemovalResult(RangeCacheRemovalCounters counters) { + Pair result = Pair.of(counters.removedEntries, counters.removedSize); + counters.recycle(); + return result; + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java index fdcff97bd4554..f2bd2c39d4bbd 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java @@ -45,7 +45,6 @@ import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; -import org.apache.bookkeeper.mledger.util.RangeCache; import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,6 +53,18 @@ * Cache data payload for entries of all ledgers. */ public class RangeEntryCacheImpl implements EntryCache { + /** + * The Netty allocator used when managedLedgerCacheCopyEntries=true. + */ + public static final PooledByteBufAllocator ALLOCATOR = new PooledByteBufAllocator(true, // preferDirect + 0, // nHeapArenas, + PooledByteBufAllocator.defaultNumDirectArena(), // nDirectArena + PooledByteBufAllocator.defaultPageSize(), // pageSize + PooledByteBufAllocator.defaultMaxOrder(), // maxOrder + PooledByteBufAllocator.defaultSmallCacheSize(), // smallCacheSize + PooledByteBufAllocator.defaultNormalCacheSize(), // normalCacheSize, + true // Use cache for all threads + ); /** * Overhead per-entry to take into account the envelope. @@ -65,7 +76,7 @@ public class RangeEntryCacheImpl implements EntryCache { private final RangeEntryCacheManagerImpl manager; final ManagedLedgerImpl ml; private ManagedLedgerInterceptor interceptor; - private final RangeCache entries; + private final RangeCache entries; private final boolean copyEntries; private final PendingReadsManager pendingReadsManager; @@ -74,12 +85,14 @@ public class RangeEntryCacheImpl implements EntryCache { private final LongAdder totalAddedEntriesSize = new LongAdder(); private final LongAdder totalAddedEntriesCount = new LongAdder(); - public RangeEntryCacheImpl(RangeEntryCacheManagerImpl manager, ManagedLedgerImpl ml, boolean copyEntries) { + public RangeEntryCacheImpl(RangeEntryCacheManagerImpl manager, ManagedLedgerImpl ml, boolean copyEntries, + RangeCacheRemovalQueue rangeCacheRemovalQueue) { + this.manager = manager; this.ml = ml; this.pendingReadsManager = new PendingReadsManager(this); this.interceptor = ml.getManagedLedgerInterceptor(); - this.entries = new RangeCache<>(EntryImpl::getLength, EntryImpl::getTimestamp); + this.entries = new RangeCache(rangeCacheRemovalQueue); this.copyEntries = copyEntries; if (log.isDebugEnabled()) { @@ -107,26 +120,9 @@ public InflightReadsLimiter getPendingReadsLimiter() { return manager.getInflightReadsLimiter(); } - public static final PooledByteBufAllocator ALLOCATOR = new PooledByteBufAllocator(true, // preferDirect - 0, // nHeapArenas, - PooledByteBufAllocator.defaultNumDirectArena(), // nDirectArena - PooledByteBufAllocator.defaultPageSize(), // pageSize - PooledByteBufAllocator.defaultMaxOrder(), // maxOrder - PooledByteBufAllocator.defaultSmallCacheSize(), // smallCacheSize - PooledByteBufAllocator.defaultNormalCacheSize(), // normalCacheSize, - true // Use cache for all threads - ); - @Override - public boolean insert(EntryImpl entry) { + public boolean insert(Entry entry) { int entryLength = entry.getLength(); - if (!manager.hasSpaceInCache()) { - if (log.isDebugEnabled()) { - log.debug("[{}] Skipping cache while doing eviction: {} - size: {}", ml.getName(), entry.getPosition(), - entryLength); - } - return false; - } if (log.isDebugEnabled()) { log.debug("[{}] Adding entry to cache: {} - size: {}", ml.getName(), entry.getPosition(), @@ -135,6 +131,7 @@ public boolean insert(EntryImpl entry) { Position position = entry.getPosition(); if (entries.exists(position)) { + // If the entry is already in the cache, don't insert it again return false; } @@ -149,7 +146,8 @@ public boolean insert(EntryImpl entry) { cachedData = entry.getDataBuffer().retain(); } - EntryImpl cacheEntry = EntryImpl.create(position, cachedData); + CachedEntryImpl cacheEntry = + CachedEntryImpl.create(position, cachedData); cachedData.release(); if (entries.put(position, cacheEntry)) { totalAddedEntriesSize.add(entryLength); @@ -163,7 +161,7 @@ public boolean insert(EntryImpl entry) { } } - private ByteBuf copyEntry(EntryImpl entry) { + private ByteBuf copyEntry(Entry entry) { // Copy the entry into a buffer owned by the cache. The reason is that the incoming entry is retaining a buffer // from netty, usually allocated in 64Kb chunks. So if we just retain the entry without copying it, we might // retain actually the full 64Kb even for a small entry @@ -376,9 +374,9 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx2) { void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, boolean shouldCacheEntry, final ReadEntriesCallback callback, Object ctx) { - Collection cachedEntries; + Collection cachedEntries; if (firstPosition.compareTo(lastPosition) == 0) { - EntryImpl cachedEntry = entries.get(firstPosition); + CachedEntry cachedEntry = entries.get(firstPosition); if (cachedEntry == null) { cachedEntries = Collections.emptyList(); } else { @@ -393,7 +391,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio final List entriesToReturn = new ArrayList<>(numberOfEntries); // All entries found in cache - for (EntryImpl entry : cachedEntries) { + for (Entry entry : cachedEntries) { entriesToReturn.add(EntryImpl.create(entry)); totalCachedSize += entry.getLength(); entry.release(); @@ -408,6 +406,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio callback.readEntriesComplete(entriesToReturn, ctx); } else { + // TODO: consider reusing the partially cached entries and only reading the missing ones if (!cachedEntries.isEmpty()) { cachedEntries.forEach(entry -> entry.release()); } @@ -441,10 +440,10 @@ private long getAvgEntrySize() { * @param shouldCacheEntry if we should put the entry into the cache * @return a handle to the operation */ - CompletableFuture> readFromStorage(ReadHandle lh, - long firstEntry, long lastEntry, boolean shouldCacheEntry) { + CompletableFuture> readFromStorage(ReadHandle lh, long firstEntry, long lastEntry, + boolean shouldCacheEntry) { final int entriesToRead = (int) (lastEntry - firstEntry) + 1; - CompletableFuture> readResult = ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry) + CompletableFuture> readResult = ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry) .thenApply( ledgerEntries -> { requireNonNull(ml.getName()); @@ -453,15 +452,13 @@ CompletableFuture> readFromStorage(ReadHandle lh, try { // We got the entries, we need to transform them to a List<> type long totalSize = 0; - final List entriesToReturn = new ArrayList<>(entriesToRead); + final List entriesToReturn = new ArrayList<>(entriesToRead); for (LedgerEntry e : ledgerEntries) { - EntryImpl entry = RangeEntryCacheManagerImpl.create(e, interceptor); + EntryImpl entry = EntryImpl.create(e, interceptor); entriesToReturn.add(entry); totalSize += entry.getLength(); if (shouldCacheEntry) { - EntryImpl cacheEntry = EntryImpl.create(entry); - insert(cacheEntry); - cacheEntry.release(); + insert(entry); } } @@ -499,32 +496,5 @@ public long getSize() { return entries.getSize(); } - @Override - public int compareTo(EntryCache other) { - return Long.compare(getSize(), other.getSize()); - } - - @Override - public Pair evictEntries(long sizeToFree) { - checkArgument(sizeToFree > 0); - Pair evicted = entries.evictLeastAccessedEntries(sizeToFree); - int evictedEntries = evicted.getLeft(); - long evictedSize = evicted.getRight(); - if (log.isDebugEnabled()) { - log.debug( - "[{}] Doing cache eviction of at least {} Mb -- Deleted {} entries - Total size deleted: {} Mb " - + " -- Current Size: {} Mb", - ml.getName(), sizeToFree / MB, evictedEntries, evictedSize / MB, entries.getSize() / MB); - } - manager.entriesRemoved(evictedSize, evictedEntries); - return evicted; - } - - @Override - public void invalidateEntriesBeforeTimestamp(long timestamp) { - Pair evictedPair = entries.evictLEntriesBeforeTimestamp(timestamp); - manager.entriesRemoved(evictedPair.getRight(), evictedPair.getLeft()); - } - private static final Logger log = LoggerFactory.getLogger(RangeEntryCacheImpl.class); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerEvictionHandler.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerEvictionHandler.java new file mode 100644 index 0000000000000..ff408bbca765b --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerEvictionHandler.java @@ -0,0 +1,67 @@ +/* + * 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.cache; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl.MB; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.tuple.Pair; + +@Slf4j +class RangeEntryCacheManagerEvictionHandler { + private final RangeEntryCacheManagerImpl manager; + private final RangeCacheRemovalQueue rangeCacheRemovalQueue; + + public RangeEntryCacheManagerEvictionHandler(RangeEntryCacheManagerImpl manager, + RangeCacheRemovalQueue rangeCacheRemovalQueue) { + this.manager = manager; + this.rangeCacheRemovalQueue = rangeCacheRemovalQueue; + } + + /** + * Invalidate all entries in the cache which were created before the given timestamp. + * + * @param timestamp the timestamp before which entries will be invalidated + */ + public void invalidateEntriesBeforeTimestampNanos(long timestamp) { + Pair evictedPair = rangeCacheRemovalQueue.evictLEntriesBeforeTimestamp(timestamp); + manager.entriesRemoved(evictedPair.getRight(), evictedPair.getLeft()); + } + + /** + * Force the cache to drop entries to free space. + * + * @param sizeToFree the total memory size to free + * @return a pair containing the number of entries evicted and their total size + */ + public Pair evictEntries(long sizeToFree) { + checkArgument(sizeToFree > 0); + Pair evicted = rangeCacheRemovalQueue.evictLeastAccessedEntries(sizeToFree); + int evictedEntries = evicted.getLeft(); + long evictedSize = evicted.getRight(); + if (log.isDebugEnabled()) { + log.debug( + "Doing cache eviction of at least {} Mb -- Deleted {} entries - Total size deleted: {} Mb " + + " -- Current Size: {} Mb", + sizeToFree / MB, evictedEntries, evictedSize / MB, manager.getSize() / MB); + } + manager.entriesRemoved(evictedSize, evictedEntries); + return evicted; + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java index 61d52aa3919ae..1e81d3166dcfd 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java @@ -18,24 +18,19 @@ */ package org.apache.bookkeeper.mledger.impl.cache; -import com.google.common.collect.Lists; -import io.netty.buffer.ByteBuf; import io.opentelemetry.api.OpenTelemetry; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import org.apache.bookkeeper.client.api.LedgerEntry; -import org.apache.bookkeeper.client.impl.LedgerEntryImpl; +import java.util.concurrent.atomic.AtomicReference; import org.apache.bookkeeper.common.util.OrderedScheduler; -import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; -import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryMBeanImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,9 +42,10 @@ public class RangeEntryCacheManagerImpl implements EntryCacheManager { private volatile double cacheEvictionWatermark; private final AtomicLong currentSize = new AtomicLong(0); private final ConcurrentMap caches = new ConcurrentHashMap(); - private final EntryCacheEvictionPolicy evictionPolicy; + private final RangeCacheRemovalQueue rangeCacheRemovalQueue; + private final RangeEntryCacheManagerEvictionHandler evictionHandler; - private final AtomicBoolean evictionInProgress = new AtomicBoolean(false); + private final AtomicReference> evictionInProgress = new AtomicReference<>(null); private final ManagedLedgerFactoryImpl mlFactory; protected final ManagedLedgerFactoryMBeanImpl mlFactoryMBean; @@ -69,9 +65,10 @@ public RangeEntryCacheManagerImpl(ManagedLedgerFactoryImpl factory, OrderedSched scheduledExecutor, openTelemetry); this.evictionTriggerThreshold = (long) (maxSize * evictionTriggerThresholdPercent); this.cacheEvictionWatermark = config.getCacheEvictionWatermark(); - this.evictionPolicy = new EntryCacheDefaultEvictionPolicy(); this.mlFactory = factory; this.mlFactoryMBean = factory.getMbean(); + this.rangeCacheRemovalQueue = new RangeCacheRemovalQueue(); + this.evictionHandler = new RangeEntryCacheManagerEvictionHandler(this, rangeCacheRemovalQueue); log.info("Initialized managed-ledger entry cache of {} Mb", maxSize / MB); } @@ -82,7 +79,8 @@ public EntryCache getEntryCache(ManagedLedgerImpl ml) { return new EntryCacheDisabled(ml); } - EntryCache newEntryCache = new RangeEntryCacheImpl(this, ml, mlFactory.getConfig().isCopyEntriesInCache()); + EntryCache newEntryCache = + new RangeEntryCacheImpl(this, ml, mlFactory.getConfig().isCopyEntriesInCache(), rangeCacheRemovalQueue); EntryCache currentEntryCache = caches.putIfAbsent(ml.getName(), newEntryCache); if (currentEntryCache != null) { return currentEntryCache; @@ -117,40 +115,72 @@ public void removeEntryCache(String name) { } } - boolean hasSpaceInCache() { + /** + * Trigger an eviction cycle if the cache size is over the threshold. + * + * @return when eviction is in progress or triggered, return a future that will be completed when the eviction + * cycle is completed + */ + Optional> triggerEvictionWhenNeeded() { long currentSize = this.currentSize.get(); // Trigger a single eviction in background. While the eviction is running we stop inserting entries in the cache - if (currentSize > evictionTriggerThreshold && evictionInProgress.compareAndSet(false, true)) { - mlFactory.getScheduledExecutor().execute(() -> { + if (currentSize > evictionTriggerThreshold) { + CompletableFuture evictionCompletionFuture = null; + while (evictionCompletionFuture == null) { + evictionCompletionFuture = evictionInProgress.get(); + if (evictionCompletionFuture == null) { + evictionCompletionFuture = evictionInProgress.updateAndGet( + currentValue -> currentValue == null ? new CompletableFuture<>() : null); + if (evictionCompletionFuture != null) { + triggerEvictionToMakeSpace(evictionCompletionFuture); + } + } + } + return Optional.of(evictionCompletionFuture); + } else { + return Optional.empty(); + } + } + + private void triggerEvictionToMakeSpace(CompletableFuture evictionCompletionFuture) { + mlFactory.getCacheEvictionExecutor().execute(() -> { + try { // Trigger a new cache eviction cycle to bring the used memory below the cacheEvictionWatermark // percentage limit - long sizeToEvict = currentSize - (long) (maxSize * cacheEvictionWatermark); - long startTime = System.nanoTime(); - log.info("Triggering cache eviction. total size: {} Mb -- Need to discard: {} Mb", currentSize / MB, - sizeToEvict / MB); + doEvictToWatermarkWhenOverThreshold(); + } finally { + evictionCompletionFuture.complete(null); + evictionInProgress.set(null); + } + }); + } + private void doEvictToWatermarkWhenOverThreshold() { + long currentSize = this.currentSize.get(); + if (currentSize > evictionTriggerThreshold) { + long sizeToEvict = currentSize - (long) (maxSize * cacheEvictionWatermark); + if (sizeToEvict > 0) { try { - evictionPolicy.doEviction(Lists.newArrayList(caches.values()), sizeToEvict); - + long startTime = System.nanoTime(); + log.info("Triggering cache eviction. total size: {} Mb -- Need to discard: {} Mb", currentSize / MB, + sizeToEvict / MB); + evictionHandler.evictEntries(sizeToEvict); long endTime = System.nanoTime(); - double durationMs = TimeUnit.NANOSECONDS.toMicros(endTime - startTime) / 1000.0; - + double durationMs = TimeUnit.NANOSECONDS.toMillis(endTime - startTime); log.info("Eviction completed. Removed {} Mb in {} ms", (currentSize - this.currentSize.get()) / MB, durationMs); } finally { mlFactoryMBean.recordCacheEviction(); - evictionInProgress.set(false); } - }); + } } - - return currentSize < maxSize; } void entryAdded(long size) { - mlFactoryMBean.recordCacheInsertion(); currentSize.addAndGet(size); + mlFactoryMBean.recordCacheInsertion(); + triggerEvictionWhenNeeded(); } void entriesRemoved(long size, int count) { @@ -174,33 +204,22 @@ public double getCacheEvictionWatermark() { } @Override - public void clear() { - caches.values().forEach(EntryCache::clear); - } - - public static Entry create(long ledgerId, long entryId, ByteBuf data) { - return EntryImpl.create(ledgerId, entryId, data); + public void doCacheEviction(long maxTimestamp) { + // this method is expected to be called from the cache eviction executor + CompletableFuture evictionCompletionFuture = new CompletableFuture<>(); + evictionInProgress.set(evictionCompletionFuture); + try { + evictionHandler.invalidateEntriesBeforeTimestampNanos(maxTimestamp); + doEvictToWatermarkWhenOverThreshold(); + } finally { + evictionCompletionFuture.complete(null); + evictionInProgress.set(null); + } } - public static EntryImpl create(LedgerEntry ledgerEntry, ManagedLedgerInterceptor interceptor) { - ManagedLedgerInterceptor.PayloadProcessorHandle processorHandle = null; - if (interceptor != null) { - ByteBuf duplicateBuffer = ledgerEntry.getEntryBuffer().retainedDuplicate(); - processorHandle = interceptor - .processPayloadBeforeEntryCache(duplicateBuffer); - if (processorHandle != null) { - ledgerEntry = LedgerEntryImpl.create(ledgerEntry.getLedgerId(), ledgerEntry.getEntryId(), - ledgerEntry.getLength(), processorHandle.getProcessedPayload()); - } else { - duplicateBuffer.release(); - } - } - EntryImpl returnEntry = EntryImpl.create(ledgerEntry); - if (processorHandle != null) { - processorHandle.release(); - ledgerEntry.close(); - } - return returnEntry; + @Override + public void clear() { + caches.values().forEach(EntryCache::clear); } public InflightReadsLimiter getInflightReadsLimiter() { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java deleted file mode 100644 index c1de09f10a6b0..0000000000000 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java +++ /dev/null @@ -1,525 +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.util; - -import static com.google.common.base.Preconditions.checkArgument; -import com.google.common.base.Predicate; -import io.netty.util.IllegalReferenceCountException; -import io.netty.util.Recycler; -import io.netty.util.Recycler.Handle; -import io.netty.util.ReferenceCounted; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentNavigableMap; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.StampedLock; -import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.mledger.util.RangeCache.ValueWithKeyValidation; -import org.apache.commons.lang3.tuple.Pair; - -/** - * Special type of cache where get() and delete() operations can be done over a range of keys. - * The implementation avoids locks and synchronization by relying on ConcurrentSkipListMap for storing the entries. - * Since there are no locks, it's necessary to ensure that a single entry in the cache is removed exactly once. - * Removing an entry multiple times could result in the entries of the cache being released multiple times, - * even while they are still in use. This is prevented by using a custom wrapper around the value to store in the map - * that ensures that the value is removed from the map only if the exact same instance is present in the map. - * There's also a check that ensures that the value matches the key. This is used to detect races without impacting - * consistency. - * - * @param - * Cache key. Needs to be Comparable - * @param - * Cache value - */ -@Slf4j -public class RangeCache, Value extends ValueWithKeyValidation> { - public interface ValueWithKeyValidation extends ReferenceCounted { - boolean matchesKey(T key); - } - - // Map from key to nodes inside the linked list - private final ConcurrentNavigableMap> entries; - private AtomicLong size; // Total size of values stored in cache - private final Weighter weighter; // Weighter object used to extract the size from values - private final TimestampExtractor timestampExtractor; // Extract the timestamp associated with a value - - /** - * Wrapper around the value to store in Map. This is needed to ensure that a specific instance can be removed from - * the map by calling the {@link Map#remove(Object, Object)} method. Certain race conditions could result in the - * wrong value being removed from the map. The instances of this class are recycled to avoid creating new objects. - */ - private static class EntryWrapper { - private final Handle recyclerHandle; - private static final Recycler RECYCLER = new Recycler() { - @Override - protected EntryWrapper newObject(Handle recyclerHandle) { - return new EntryWrapper(recyclerHandle); - } - }; - private final StampedLock lock = new StampedLock(); - private K key; - private V value; - long size; - - private EntryWrapper(Handle recyclerHandle) { - this.recyclerHandle = recyclerHandle; - } - - static EntryWrapper create(K key, V value, long size) { - EntryWrapper entryWrapper = RECYCLER.get(); - long stamp = entryWrapper.lock.writeLock(); - entryWrapper.key = key; - entryWrapper.value = value; - entryWrapper.size = size; - entryWrapper.lock.unlockWrite(stamp); - return entryWrapper; - } - - K getKey() { - long stamp = lock.tryOptimisticRead(); - K localKey = key; - if (!lock.validate(stamp)) { - stamp = lock.readLock(); - localKey = key; - lock.unlockRead(stamp); - } - return localKey; - } - - /** - * Get the value associated with the key. Returns null if the key does not match the key. - * - * @param key the key to match - * @return the value associated with the key, or null if the value has already been recycled or the key does not - * match - */ - V getValue(K key) { - return getValueInternal(key, false); - } - - /** - * Get the value associated with the Map.Entry's key and value. Exact instance of the key is required to match. - * @param entry the entry which contains the key and {@link EntryWrapper} value to get the value from - * @return the value associated with the key, or null if the value has already been recycled or the key does not - * exactly match the same instance - */ - static V getValueMatchingMapEntry(Map.Entry> entry) { - return entry.getValue().getValueInternal(entry.getKey(), true); - } - - /** - * Get the value associated with the key. Returns null if the key does not match the key associated with the - * value. - * - * @param key the key to match - * @param requireSameKeyInstance when true, the matching will be restricted to exactly the same instance of the - * key as the one stored in the wrapper. This is used to avoid any races - * when retrieving or removing the entries from the cache when the key and value - * instances are available. - * @return the value associated with the key, or null if the key does not match - */ - private V getValueInternal(K key, boolean requireSameKeyInstance) { - long stamp = lock.tryOptimisticRead(); - K localKey = this.key; - V localValue = this.value; - if (!lock.validate(stamp)) { - stamp = lock.readLock(); - localKey = this.key; - localValue = this.value; - lock.unlockRead(stamp); - } - // check that the given key matches the key associated with the value in the entry - // this is used to detect if the entry has already been recycled and contains another key - // when requireSameKeyInstance is true, the key must be exactly the same instance as the one stored in the - // entry to match - if (localKey != key && (requireSameKeyInstance || localKey == null || !localKey.equals(key))) { - return null; - } - return localValue; - } - - long getSize() { - long stamp = lock.tryOptimisticRead(); - long localSize = size; - if (!lock.validate(stamp)) { - stamp = lock.readLock(); - localSize = size; - lock.unlockRead(stamp); - } - return localSize; - } - - void recycle() { - key = null; - value = null; - size = 0; - recyclerHandle.recycle(this); - } - } - - /** - * Mutable object to store the number of entries and the total size removed from the cache. The instances - * are recycled to avoid creating new instances. - */ - private static class RemovalCounters { - private final Handle recyclerHandle; - private static final Recycler RECYCLER = new Recycler() { - @Override - protected RemovalCounters newObject(Handle recyclerHandle) { - return new RemovalCounters(recyclerHandle); - } - }; - int removedEntries; - long removedSize; - private RemovalCounters(Handle recyclerHandle) { - this.recyclerHandle = recyclerHandle; - } - - static RemovalCounters create() { - RemovalCounters results = RECYCLER.get(); - results.removedEntries = 0; - results.removedSize = 0; - return results; - } - - void recycle() { - removedEntries = 0; - removedSize = 0; - recyclerHandle.recycle(this); - } - - public void entryRemoved(long size) { - removedSize += size; - removedEntries++; - } - } - - /** - * Construct a new RangeLruCache with default Weighter. - */ - public RangeCache() { - this(new DefaultWeighter<>(), (x) -> System.nanoTime()); - } - - /** - * Construct a new RangeLruCache. - * - * @param weighter - * a custom weighter to compute the size of each stored value - */ - public RangeCache(Weighter weighter, TimestampExtractor timestampExtractor) { - this.size = new AtomicLong(0); - this.entries = new ConcurrentSkipListMap<>(); - this.weighter = weighter; - this.timestampExtractor = timestampExtractor; - } - - /** - * Insert. - * - * @param key - * @param value ref counted value with at least 1 ref to pass on the cache - * @return whether the entry was inserted in the cache - */ - public boolean put(Key key, Value value) { - // retain value so that it's not released before we put it in the cache and calculate the weight - value.retain(); - try { - if (!value.matchesKey(key)) { - throw new IllegalArgumentException("Value '" + value + "' does not match key '" + key + "'"); - } - long entrySize = weighter.getSize(value); - EntryWrapper newWrapper = EntryWrapper.create(key, value, entrySize); - if (entries.putIfAbsent(key, newWrapper) == null) { - this.size.addAndGet(entrySize); - return true; - } else { - // recycle the new wrapper as it was not used - newWrapper.recycle(); - return false; - } - } finally { - value.release(); - } - } - - public boolean exists(Key key) { - return key != null ? entries.containsKey(key) : true; - } - - /** - * Get the value associated with the key and increment the reference count of it. - * The caller is responsible for releasing the reference. - */ - public Value get(Key key) { - return getValueFromWrapper(key, entries.get(key)); - } - - private Value getValueFromWrapper(Key key, EntryWrapper valueWrapper) { - if (valueWrapper == null) { - return null; - } else { - Value value = valueWrapper.getValue(key); - return getRetainedValueMatchingKey(key, value); - } - } - - /** - * @apiNote the returned value must be released if it's not null - */ - private Value getValueMatchingEntry(Map.Entry> entry) { - Value valueMatchingEntry = EntryWrapper.getValueMatchingMapEntry(entry); - return getRetainedValueMatchingKey(entry.getKey(), valueMatchingEntry); - } - - // validates that the value matches the key and that the value has not been recycled - // which are possible due to the lack of exclusive locks in the cache and the use of reference counted objects - /** - * @apiNote the returned value must be released if it's not null - */ - private Value getRetainedValueMatchingKey(Key key, Value value) { - if (value == null) { - // the wrapper has been recycled and contains another key - return null; - } - try { - value.retain(); - } catch (IllegalReferenceCountException e) { - // Value was already deallocated - return null; - } - // check that the value matches the key and that there's at least 2 references to it since - // the cache should be holding one reference and a new reference was just added in this method - if (value.refCnt() > 1 && value.matchesKey(key)) { - return value; - } else { - // Value or IdentityWrapper was recycled and already contains another value - // release the reference added in this method - value.release(); - return null; - } - } - - /** - * - * @param first - * the first key in the range - * @param last - * the last key in the range (inclusive) - * @return a collections of the value found in cache - */ - public Collection getRange(Key first, Key last) { - List values = new ArrayList(); - - // Return the values of the entries found in cache - for (Map.Entry> entry : entries.subMap(first, true, last, true).entrySet()) { - Value value = getValueMatchingEntry(entry); - if (value != null) { - values.add(value); - } - } - - return values; - } - - /** - * - * @param first - * @param last - * @param lastInclusive - * @return an pair of ints, containing the number of removed entries and the total size - */ - public Pair removeRange(Key first, Key last, boolean lastInclusive) { - if (log.isDebugEnabled()) { - log.debug("Removing entries in range [{}, {}], lastInclusive: {}", first, last, lastInclusive); - } - RemovalCounters counters = RemovalCounters.create(); - Map> subMap = entries.subMap(first, true, last, lastInclusive); - for (Map.Entry> entry : subMap.entrySet()) { - removeEntry(entry, counters); - } - return handleRemovalResult(counters); - } - - enum RemoveEntryResult { - ENTRY_REMOVED, - CONTINUE_LOOP, - BREAK_LOOP; - } - - private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters) { - return removeEntry(entry, counters, x -> true); - } - - private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters, - Predicate removeCondition) { - Key key = entry.getKey(); - EntryWrapper entryWrapper = entry.getValue(); - Value value = getValueMatchingEntry(entry); - if (value == null) { - // the wrapper has already been recycled or contains another key - entries.remove(key, entryWrapper); - return RemoveEntryResult.CONTINUE_LOOP; - } - try { - if (!removeCondition.test(value)) { - return RemoveEntryResult.BREAK_LOOP; - } - // remove the specific entry - boolean entryRemoved = entries.remove(key, entryWrapper); - if (entryRemoved) { - counters.entryRemoved(entryWrapper.getSize()); - // check that the value hasn't been recycled in between - // there should be at least 2 references since this method adds one and the cache should have - // one reference. it is valid that the value contains references even after the key has been - // removed from the cache - if (value.refCnt() > 1) { - entryWrapper.recycle(); - // remove the cache reference - value.release(); - } else { - log.info("Unexpected refCnt {} for key {}, removed entry without releasing the value", - value.refCnt(), key); - } - return RemoveEntryResult.ENTRY_REMOVED; - } else { - return RemoveEntryResult.CONTINUE_LOOP; - } - } finally { - // remove the extra retain - value.release(); - } - } - - private Pair handleRemovalResult(RemovalCounters counters) { - size.addAndGet(-counters.removedSize); - Pair result = Pair.of(counters.removedEntries, counters.removedSize); - counters.recycle(); - return result; - } - - /** - * - * @param minSize - * @return a pair containing the number of entries evicted and their total size - */ - public Pair evictLeastAccessedEntries(long minSize) { - if (log.isDebugEnabled()) { - log.debug("Evicting entries to reach a minimum size of {}", minSize); - } - checkArgument(minSize > 0); - RemovalCounters counters = RemovalCounters.create(); - while (counters.removedSize < minSize && !Thread.currentThread().isInterrupted()) { - Map.Entry> entry = entries.firstEntry(); - if (entry == null) { - break; - } - removeEntry(entry, counters); - } - return handleRemovalResult(counters); - } - - /** - * - * @param maxTimestamp the max timestamp of the entries to be evicted - * @return the tota - */ - public Pair evictLEntriesBeforeTimestamp(long maxTimestamp) { - if (log.isDebugEnabled()) { - log.debug("Evicting entries with timestamp <= {}", maxTimestamp); - } - RemovalCounters counters = RemovalCounters.create(); - while (!Thread.currentThread().isInterrupted()) { - Map.Entry> entry = entries.firstEntry(); - if (entry == null) { - break; - } - if (removeEntry(entry, counters, value -> timestampExtractor.getTimestamp(value) <= maxTimestamp) - == RemoveEntryResult.BREAK_LOOP) { - break; - } - } - return handleRemovalResult(counters); - } - - /** - * Just for testing. Getting the number of entries is very expensive on the conncurrent map - */ - protected long getNumberOfEntries() { - return entries.size(); - } - - public long getSize() { - return size.get(); - } - - /** - * Remove all the entries from the cache. - * - * @return size of removed entries - */ - public Pair clear() { - if (log.isDebugEnabled()) { - log.debug("Clearing the cache with {} entries and size {}", entries.size(), size.get()); - } - RemovalCounters counters = RemovalCounters.create(); - while (!Thread.currentThread().isInterrupted()) { - Map.Entry> entry = entries.firstEntry(); - if (entry == null) { - break; - } - removeEntry(entry, counters); - } - return handleRemovalResult(counters); - } - - /** - * Interface of a object that is able to the extract the "weight" (size/cost/space) of the cached values. - * - * @param - */ - public interface Weighter { - long getSize(ValueT value); - } - - /** - * Interface of a object that is able to the extract the "timestamp" of the cached values. - * - * @param - */ - public interface TimestampExtractor { - long getTimestamp(ValueT value); - } - - /** - * Default cache weighter, every value is assumed the same cost. - * - * @param - */ - private static class DefaultWeighter implements Weighter { - @Override - public long getSize(Value value) { - return 1; - } - } -} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java index f00efb27ca5ab..3a0a8b2a52a99 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java @@ -20,19 +20,17 @@ import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; - import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; - import lombok.Cleanup; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.mledger.AsyncCallbacks; @@ -48,6 +46,7 @@ import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.Test; @@ -77,6 +76,8 @@ public void simple() throws Exception { ManagedLedgerFactoryConfig config = new ManagedLedgerFactoryConfig(); config.setMaxCacheSize(10); config.setCacheEvictionWatermark(0.8); + config.setCacheEvictionIntervalMs(1000); + config.setCacheEvictionTimeThresholdMillis(1000); @Cleanup("shutdown") ManagedLedgerFactoryImpl factory2 = new ManagedLedgerFactoryImpl(metadataStore, bkc, config); @@ -113,31 +114,31 @@ public void simple() throws Exception { // The algorithm should evict entries from cache1 cache2.insert(EntryImpl.create(2, 3, new byte[1])); - // Wait for eviction to be completed in background - Thread.sleep(100); + factory2.waitForPendingCacheEvictions(); + assertEquals(cacheManager.getSize(), 7); - assertEquals(cache1.getSize(), 4); - assertEquals(cache2.getSize(), 3); + assertEquals(cache1.getSize(), 3); + assertEquals(cache2.getSize(), 4); cacheManager.removeEntryCache("cache1"); - assertEquals(cacheManager.getSize(), 3); - assertEquals(cache2.getSize(), 3); + assertEquals(cacheManager.getSize(), 4); + assertEquals(cache2.getSize(), 4); // Should remove 1 entry cache2.invalidateEntries(PositionFactory.create(2, 1)); - assertEquals(cacheManager.getSize(), 2); - assertEquals(cache2.getSize(), 2); + assertEquals(cacheManager.getSize(), 3); + assertEquals(cache2.getSize(), 3); factory2.getMbean().refreshStats(1, TimeUnit.SECONDS); assertEquals(factory2.getMbean().getCacheMaxSize(), 10); - assertEquals(factory2.getMbean().getCacheUsedSize(), 2); + assertEquals(factory2.getMbean().getCacheUsedSize(), 3); assertEquals(factory2.getMbean().getCacheHitsRate(), 0.0); assertEquals(factory2.getMbean().getCacheMissesRate(), 0.0); assertEquals(factory2.getMbean().getCacheHitsThroughput(), 0.0); assertEquals(factory2.getMbean().getNumberOfCacheEvictions(), 1); - assertEquals(factory2.getMbean().getCacheInsertedEntriesCount(), 5); - assertEquals(factory2.getMbean().getCacheEntriesCount(), 2); + assertEquals(factory2.getMbean().getCacheInsertedEntriesCount(), 6); + assertEquals(factory2.getMbean().getCacheEntriesCount(), 3); assertEquals(factory2.getMbean().getCacheEvictedEntriesCount(), 3); } @@ -173,6 +174,8 @@ public void cacheSizeUpdate() throws Exception { ManagedLedgerFactoryConfig config = new ManagedLedgerFactoryConfig(); config.setMaxCacheSize(200); config.setCacheEvictionWatermark(0.8); + // Set a long eviction time threshold to avoid eviction by timestamp during the test + config.setCacheEvictionTimeThresholdMillis(20000); @Cleanup("shutdown") ManagedLedgerFactoryImpl factory2 = new ManagedLedgerFactoryImpl(metadataStore, bkc, config); @@ -186,7 +189,12 @@ public void cacheSizeUpdate() throws Exception { entries.add(EntryImpl.create(1, i, new byte[i + 1])); assertTrue(cache1.insert(entries.get(i))); } - assertEquals(210, cacheManager.getSize()); + + // cache eviction gets triggered asynchronously + Awaitility.await().untilAsserted(() -> { + // cache size should be less or equal to 0.8 (cacheEvictionWatermark) * 200 (maxCacheSize) + assertEquals(cacheManager.getSize(), 155); + }); // Consume some entries. Random random = new Random(); @@ -199,7 +207,7 @@ public void cacheSizeUpdate() throws Exception { cacheManager.removeEntryCache(ml1.getName()); assertEquals(factory2.getMbean().getCacheInsertedEntriesCount(), 20); assertEquals(factory2.getMbean().getCacheEntriesCount(), 0); - assertEquals(0, cacheManager.getSize()); + assertEquals(cacheManager.getSize(), 0); assertEquals(factory2.getMbean().getCacheEvictedEntriesCount(), 20); } @@ -280,15 +288,18 @@ public void verifyNoCacheIfNoConsumer() throws Exception { @Test public void verifyHitsMisses() throws Exception { ManagedLedgerFactoryConfig config = new ManagedLedgerFactoryConfig(); - config.setMaxCacheSize(7 * 10); + config.setMaxCacheSize(100); config.setCacheEvictionWatermark(0.8); config.setCacheEvictionIntervalMs(1000); + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setCacheEvictionByMarkDeletedPosition(false); + @Cleanup("shutdown") ManagedLedgerFactoryImpl factory2 = new ManagedLedgerFactoryImpl(metadataStore, bkc, config); EntryCacheManager cacheManager = factory2.getEntryCacheManager(); - ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory2.open("ledger"); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory2.open("ledger", managedLedgerConfig); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); ManagedCursorImpl c2 = (ManagedCursorImpl) ledger.openCursor("c2"); @@ -327,6 +338,8 @@ public void verifyHitsMisses() throws Exception { entries = c2.readEntries(10); assertEquals(entries.size(), 10); + Thread.sleep(200L); + factory2.getMbean().refreshStats(1, TimeUnit.SECONDS); assertEquals(factory2.getMbean().getCacheUsedSize(), 0); assertEquals(factory2.getMbean().getCacheHitsRate(), 10.0); @@ -338,6 +351,8 @@ public void verifyHitsMisses() throws Exception { c2.setReadPosition(pos); entries.forEach(Entry::release); + Thread.sleep(200L); + factory2.getMbean().refreshStats(1, TimeUnit.SECONDS); assertEquals(factory2.getMbean().getCacheUsedSize(), 0); assertEquals(factory2.getMbean().getCacheHitsRate(), 0.0); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java index 6676baf8b555a..92726b5838b24 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java @@ -93,7 +93,7 @@ public void testPreciseLimitation(String missingCase) throws Exception { ml.addEntry(new byte[]{i}); } // Evict cached entries. - entryCache.evictEntries(ml.currentLedgerSize); + entryCache.clear(); Assert.assertEquals(entryCache.getSize(), 0); CountDownLatch readCompleteSignal1 = new CountDownLatch(1); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index 2afbcef0926e7..d2fd2c8d063c3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -36,6 +36,8 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.ClearBacklogCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; @@ -58,13 +60,24 @@ public class ManagedCursorContainerTest { private static class MockManagedCursor implements ManagedCursor { ManagedCursorContainer container; - Position position; + Position markDeletePosition; + Position readPosition; + private final boolean updateMarkDeletePosition; + private final boolean durable; String name; - public MockManagedCursor(ManagedCursorContainer container, String name, Position position) { + public MockManagedCursor(ManagedCursorContainer container, String name, Position markDeletePosition) { + this(container, name, markDeletePosition, null, true, true); + } + + public MockManagedCursor(ManagedCursorContainer container, String name, Position markDeletePosition, + Position readPosition, boolean updateMarkDeletePosition, boolean durable) { this.container = container; this.name = name; - this.position = position; + this.markDeletePosition = markDeletePosition; + this.readPosition = readPosition; + this.updateMarkDeletePosition = updateMarkDeletePosition; + this.durable = durable; } @Override @@ -104,7 +117,7 @@ public boolean removeProperty(String key) { @Override public boolean isDurable() { - return true; + return durable; } @Override @@ -146,8 +159,10 @@ public void markDelete(Position position) { @Override public void markDelete(Position position, Map properties) { - this.position = position; - container.cursorUpdated(this, position); + this.markDeletePosition = position; + if (updateMarkDeletePosition) { + container.cursorUpdated(this, position); + } } @Override @@ -163,12 +178,12 @@ public void asyncMarkDelete(Position position, Map properties, Mar @Override public Position getMarkDeletedPosition() { - return position; + return markDeletePosition; } @Override public Position getPersistentMarkDeletedPosition() { - return position; + return markDeletePosition; } @Override @@ -187,12 +202,12 @@ public void updateLastActive() { } public String toString() { - return String.format("%s=%s", name, position); + return String.format("%s=%s/%s", name, markDeletePosition, readPosition); } @Override public Position getReadPosition() { - return null; + return readPosition; } @Override @@ -201,6 +216,10 @@ public void rewind() { @Override public void seek(Position newReadPosition, boolean force) { + this.readPosition = newReadPosition; + if (!updateMarkDeletePosition) { + container.cursorUpdated(this, newReadPosition); + } } @Override @@ -441,6 +460,8 @@ public int applyMaxSizeCap(int maxEntries, long maxSizeBytes) { public void updateReadStats(int readEntriesCount, long readEntriesSize) { } + + } @Test @@ -504,7 +525,7 @@ public void simple() throws Exception { assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), cursor3, PositionFactory.create(2, 0)); - assertEquals(container.toString(), "[test1=5:5, test2=2:2, test3=2:0]"); + assertEquals(container.toString(), "[test1=5:5/null, test2=2:2/null, test3=2:0/null]"); ManagedCursor cursor4 = new MockManagedCursor(container, "test4", PositionFactory.create(4, 0)); container.add(cursor4, cursor4.getMarkDeletedPosition()); @@ -543,7 +564,7 @@ public void simple() throws Exception { container.add(cursor6, cursor6.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(6, 5)); - assertEquals(container.toString(), "[test6=6:5]"); + assertEquals(container.toString(), "[test6=6:5/null]"); } @Test @@ -558,12 +579,12 @@ public void updatingCursorOutsideContainer() { container.add(cursor2, cursor2.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 2)); - cursor2.position = PositionFactory.create(8, 8); + cursor2.markDeletePosition = PositionFactory.create(8, 8); // Until we don't update the container, the ordering will not change assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 2)); - container.cursorUpdated(cursor2, cursor2.position); + container.cursorUpdated(cursor2, cursor2.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 5)); assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), @@ -666,32 +687,32 @@ public void orderingWithUpdates() { assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c1.position = PositionFactory.create(5, 8); - container.cursorUpdated(c1, c1.position); + c1.markDeletePosition = PositionFactory.create(5, 8); + container.cursorUpdated(c1, c1.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c2.position = PositionFactory.create(5, 6); - container.cursorUpdated(c2, c2.position); + c2.markDeletePosition = PositionFactory.create(5, 6); + container.cursorUpdated(c2, c2.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c1.position = PositionFactory.create(6, 8); - container.cursorUpdated(c1, c1.position); + c1.markDeletePosition = PositionFactory.create(6, 8); + container.cursorUpdated(c1, c1.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c3.position = PositionFactory.create(8, 5); - container.cursorUpdated(c3, c3.position); + c3.markDeletePosition = PositionFactory.create(8, 5); + container.cursorUpdated(c3, c3.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c1.position = PositionFactory.create(8, 4); - container.cursorUpdated(c1, c1.position); + c1.markDeletePosition = PositionFactory.create(8, 4); + container.cursorUpdated(c1, c1.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c2.position = PositionFactory.create(8, 4); - container.cursorUpdated(c2, c2.position); + c2.markDeletePosition = PositionFactory.create(8, 4); + container.cursorUpdated(c2, c2.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(6, 4)); - c4.position = PositionFactory.create(7, 1); - container.cursorUpdated(c4, c4.position); + c4.markDeletePosition = PositionFactory.create(7, 1); + container.cursorUpdated(c4, c4.markDeletePosition); // //// @@ -731,32 +752,32 @@ public void orderingWithUpdatesAndReset() { assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c1.position = PositionFactory.create(5, 8); - container.cursorUpdated(c1, c1.position); + c1.markDeletePosition = PositionFactory.create(5, 8); + container.cursorUpdated(c1, c1.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c1.position = PositionFactory.create(5, 6); - container.cursorUpdated(c1, c1.position); + c1.markDeletePosition = PositionFactory.create(5, 6); + container.cursorUpdated(c1, c1.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c2.position = PositionFactory.create(6, 8); - container.cursorUpdated(c2, c2.position); + c2.markDeletePosition = PositionFactory.create(6, 8); + container.cursorUpdated(c2, c2.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c3.position = PositionFactory.create(8, 5); - container.cursorUpdated(c3, c3.position); + c3.markDeletePosition = PositionFactory.create(8, 5); + container.cursorUpdated(c3, c3.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c1.position = PositionFactory.create(8, 4); - container.cursorUpdated(c1, c1.position); + c1.markDeletePosition = PositionFactory.create(8, 4); + container.cursorUpdated(c1, c1.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(6, 4)); - c2.position = PositionFactory.create(4, 4); - container.cursorUpdated(c2, c2.position); + c2.markDeletePosition = PositionFactory.create(4, 4); + container.cursorUpdated(c2, c2.markDeletePosition); assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(4, 4)); - c4.position = PositionFactory.create(7, 1); - container.cursorUpdated(c4, c4.position); + c4.markDeletePosition = PositionFactory.create(7, 1); + container.cursorUpdated(c4, c4.markDeletePosition); // //// @@ -829,4 +850,23 @@ public void testVersions() { // newVersion > version assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); } + + @Test + public void testSlowestReader() { + // test 100 times + for (int i = 0; i < 100; i++) { + ManagedCursorContainer container = new ManagedCursorContainer(); + List cursors = IntStream.rangeClosed(1, 100) + .mapToObj(idx -> createCursor(container, "cursor" + idx, PositionFactory.create(0, idx))) + .collect(Collectors.toList()); + // randomize adding order + Collections.shuffle(cursors); + cursors.forEach(cursor -> container.add(cursor, cursor.getReadPosition())); + assertEquals(container.getSlowestReader().getName(), "cursor1"); + } + } + + private static ManagedCursor createCursor(ManagedCursorContainer container, String name, Position position) { + return new MockManagedCursor(container, name, position, position, false, true); + } } 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 ce8b0334226cf..3be49f584c13d 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 @@ -2892,15 +2892,13 @@ void testTrimDeletedEntries() throws ManagedLedgerException, InterruptedExceptio c1.delete(Lists.newArrayList(p1, p2, p3, p4)); - EntryImpl entry1 = EntryImpl.create(p1, ByteBufAllocator.DEFAULT.buffer(0)); - EntryImpl entry2 = EntryImpl.create(p2, ByteBufAllocator.DEFAULT.buffer(0)); - EntryImpl entry3 = EntryImpl.create(p3, ByteBufAllocator.DEFAULT.buffer(0)); - EntryImpl entry4 = EntryImpl.create(p4, ByteBufAllocator.DEFAULT.buffer(0)); - EntryImpl entry5 = EntryImpl.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 7, - ByteBufAllocator.DEFAULT.buffer(0)); + EntryImpl entry1 = createEntry(p1); + EntryImpl entry2 = createEntry(p2); + EntryImpl entry3 = createEntry(p3); + EntryImpl entry4 = createEntry(p4); + EntryImpl entry5 = createEntry( + PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 7)); List entries = Lists.newArrayList(entry1, entry2, entry3, entry4, entry5); - // release data buffers since EntryImpl.create will retain the buffer - entries.forEach(entry -> entry.getDataBuffer().release()); c1.trimDeletedEntries(entries); assertEquals(entries.size(), 1); @@ -2916,6 +2914,16 @@ void testTrimDeletedEntries() throws ManagedLedgerException, InterruptedExceptio entries.forEach(Entry::release); } + private static EntryImpl createEntry(Position p1) { + return createEntryAndReleaseBuffer(p1, ByteBufAllocator.DEFAULT.buffer(0)); + } + + private static EntryImpl createEntryAndReleaseBuffer(Position p1, ByteBuf buffer) { + EntryImpl entry = EntryImpl.create(p1, buffer); + buffer.release(); + return entry; + } + @Test(timeOut = 20000) void outOfOrderAcks() throws Exception { ManagedLedger ledger = factory.open("outOfOrderAcks"); 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 bb462e922e7e8..e8ab8e55c39a9 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 @@ -434,6 +434,7 @@ public void shouldKeepEntriesInCacheByEarliestReadPosition() throws ManagedLedge // This test case reproduces issue #16054 ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); factory.updateCacheEvictionTimeThreshold(TimeUnit.MILLISECONDS .toNanos(30000)); @@ -489,6 +490,7 @@ public void shouldKeepEntriesInCacheByEarliestMarkDeletePosition() throws Manage // This test case reproduces issue #16054 ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setCacheEvictionByMarkDeletedPosition(true); factory.updateCacheEvictionTimeThreshold(TimeUnit.MILLISECONDS .toNanos(30000)); @@ -550,7 +552,8 @@ public void shouldKeepEntriesInCacheByEarliestMarkDeletePosition() throws Manage public void asyncAPI() throws Throwable { final CountDownLatch counter = new CountDownLatch(1); - factory.asyncOpen("my_test_ledger", new ManagedLedgerConfig(), new OpenLedgerCallback() { + factory.asyncOpen("my_test_ledger", initManagedLedgerConfig(new ManagedLedgerConfig()), + new OpenLedgerCallback() { @Override public void openLedgerComplete(ManagedLedger ledger, Object ctx) { ledger.asyncOpenCursor("test-cursor", new OpenCursorCallback() { @@ -633,7 +636,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { @Test(timeOut = 20000) public void spanningMultipleLedgers() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(10); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(10); ManagedLedger ledger = factory.open("my_test_ledger", config); assertEquals(ledger.getNumberOfEntries(), 0); @@ -694,7 +697,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { @Test(timeOut = 20000) public void spanningMultipleLedgersWithSize() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(1000000); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1000000); config.setMaxSizePerLedgerMb(1); config.setEnsembleSize(1); config.setWriteQuorumSize(1).setAckQuorumSize(1); @@ -813,7 +816,8 @@ public void asyncDeleteWithError() throws Exception { stopMetadataStore(); // Delete and reopen - factory.open("my_test_ledger", new ManagedLedgerConfig()).asyncDelete(new DeleteLedgerCallback() { + factory.open("my_test_ledger", initManagedLedgerConfig(new ManagedLedgerConfig())) + .asyncDelete(new DeleteLedgerCallback() { @Override public void deleteLedgerComplete(Object ctx) { @@ -842,7 +846,7 @@ private byte[] copyBytesFromByteBuf(final ByteBuf buf) { @Test(timeOut = 20000) public void asyncAddEntryWithoutError() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", - new ManagedLedgerConfig().setMaxEntriesPerLedger(2)); + initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(2)); ledger.openCursor("test-cursor"); final int count = 4; @@ -1012,7 +1016,7 @@ public void openCursorFailed(ManagedLedgerException exception, Object ctx) { @Test(timeOut = 20000) public void readFromOlderLedger() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(1); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1); ManagedLedger ledger = factory.open("my_test_ledger", config); ManagedCursor cursor = ledger.openCursor("test"); @@ -1024,7 +1028,7 @@ public void readFromOlderLedger() throws Exception { @Test(timeOut = 20000) public void readFromOlderLedgers() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(1); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1); ManagedLedger ledger = factory.open("my_test_ledger", config); ManagedCursor cursor = ledger.openCursor("test"); @@ -1044,7 +1048,7 @@ public void readFromOlderLedgers() throws Exception { @Test(timeOut = 20000) public void triggerLedgerDeletion() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(1); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1); ManagedLedger ledger = factory.open("my_test_ledger", config); ManagedCursor cursor = ledger.openCursor("test"); @@ -1079,7 +1083,7 @@ public void testEmptyManagedLedgerContent() throws Exception { @Test(timeOut = 20000) public void testProducerAndNoConsumer() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(1); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1); ManagedLedger ledger = factory.open("my_test_ledger", config); assertEquals(ledger.getNumberOfEntries(), 0); @@ -1104,7 +1108,7 @@ public void testProducerAndNoConsumer() throws Exception { @Test(timeOut = 20000) public void testTrimmer() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(1); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1); ManagedLedger ledger = factory.open("my_test_ledger", config); ManagedCursor cursor = ledger.openCursor("c1"); @@ -1137,7 +1141,7 @@ public void testTrimmer() throws Exception { @Test(timeOut = 20000) public void testAsyncAddEntryAndSyncClose() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(10); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(10); ManagedLedger ledger = factory.open("my_test_ledger", config); ledger.openCursor("c1"); @@ -1168,7 +1172,7 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { @Test(timeOut = 20000) public void moveCursorToNextLedger() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(1); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1); ManagedLedger ledger = factory.open("my_test_ledger", config); ManagedCursor cursor = ledger.openCursor("test"); @@ -1287,7 +1291,7 @@ public void fenceManagedLedger() throws Exception { @Test public void forceCloseLedgers() throws Exception { - ManagedLedger ledger1 = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(1)); + ManagedLedger ledger1 = factory.open("my_test_ledger", initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1)); ledger1.openCursor("c1"); ManagedCursor c2 = ledger1.openCursor("c2"); ledger1.addEntry("entry-1".getBytes(Encoding)); @@ -1363,7 +1367,7 @@ public void deleteWithErrors2() throws Exception { @Test(timeOut = 20000) public void readWithErrors1() throws Exception { - ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(1)); + ManagedLedger ledger = factory.open("my_test_ledger", initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1)); ManagedCursor cursor = ledger.openCursor("c1"); ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); @@ -1568,7 +1572,7 @@ public void updatePropertiesFailed(ManagedLedgerException exception, Object ctx) @Test public void testConcurrentAsyncSetProperties() throws Exception { final CountDownLatch latch = new CountDownLatch(1000); - ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(1)); + ManagedLedger ledger = factory.open("my_test_ledger", initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1)); @Cleanup("shutdownNow") ExecutorService executor = Executors.newCachedThreadPool(); for (int i = 0; i < 1000; i++) { @@ -1676,7 +1680,7 @@ public void testReopenAndCleanup() throws Exception { assertEquals(bkc.getLedgers().size(), 2); ledger.close(); - factory.open("my_test_ledger", new ManagedLedgerConfig()).delete(); + factory.open("my_test_ledger", initManagedLedgerConfig(new ManagedLedgerConfig())).delete(); Thread.sleep(100); assertEquals(bkc.getLedgers().size(), 0); @@ -1700,7 +1704,7 @@ public void compositeNames() throws Exception { @Test public void previousPosition() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", - new ManagedLedgerConfig().setMaxEntriesPerLedger(2)); + initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(2)); ManagedCursor cursor = ledger.openCursor("my_cursor"); Position p0 = cursor.getMarkDeletedPosition(); @@ -1711,18 +1715,18 @@ public void previousPosition() throws Exception { ledger.close(); ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", - new ManagedLedgerConfig().setMaxEntriesPerLedger(2)); + initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(2)); // again ledger.close(); ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", - new ManagedLedgerConfig().setMaxEntriesPerLedger(2)); + initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(2)); Position pBeforeWriting = ledger.getLastPosition(); Position p1 = ledger.addEntry("entry".getBytes()); ledger.close(); ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", - new ManagedLedgerConfig().setMaxEntriesPerLedger(2)); + initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(2)); Position p2 = ledger.addEntry("entry".getBytes()); Position p3 = ledger.addEntry("entry".getBytes()); Position p4 = ledger.addEntry("entry".getBytes()); @@ -1739,6 +1743,7 @@ public void previousPosition() throws Exception { @Test(timeOut = 20000) public void testOpenRaceCondition() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setEnsembleSize(2).setAckQuorumSize(2).setMetadataEnsembleSize(2); final ManagedLedger ledger = factory.open("my-ledger", config); final ManagedCursor c1 = ledger.openCursor("c1"); @@ -1776,10 +1781,8 @@ public void testOpenRaceCondition() throws Exception { @Test public void invalidateConsumedEntriesFromCache() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig(); ManagedLedgerImpl ledger = - (ManagedLedgerImpl) factory.open("my_test_ledger_for_invalidateConsumedEntriesFromCache", - config); + (ManagedLedgerImpl) factory.open("my_test_ledger_for_invalidateConsumedEntriesFromCache"); EntryCacheManager cacheManager = factory.getEntryCacheManager(); EntryCache entryCache = ledger.entryCache; @@ -1799,26 +1802,41 @@ public void invalidateConsumedEntriesFromCache() throws Exception { c2.setReadPosition(p3); + + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7 * 4); assertEquals(cacheManager.getSize(), entryCache.getSize()); c1.setReadPosition(p2); + + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7 * 3); assertEquals(cacheManager.getSize(), entryCache.getSize()); c1.setReadPosition(p3); + + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7 * 2); assertEquals(cacheManager.getSize(), entryCache.getSize()); + ledger.deactivateCursor(c1); + + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7 * 2); // as c2.readPosition=p3 => Cache contains p3,p4 assertEquals(cacheManager.getSize(), entryCache.getSize()); + c2.setReadPosition(p4); + + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7); assertEquals(cacheManager.getSize(), entryCache.getSize()); + ledger.deactivateCursor(c2); + + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 0); assertEquals(cacheManager.getSize(), entryCache.getSize()); } @@ -1826,6 +1844,7 @@ public void invalidateConsumedEntriesFromCache() throws Exception { @Test public void invalidateEntriesFromCacheByMarkDeletePosition() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setCacheEvictionByMarkDeletedPosition(true); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger_for_invalidateEntriesFromCacheByMarkDeletePosition", @@ -1850,28 +1869,42 @@ public void invalidateEntriesFromCacheByMarkDeletePosition() throws Exception { c2.setReadPosition(p4); c2.markDelete(p3); + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7 * 4); assertEquals(cacheManager.getSize(), entryCache.getSize()); + c1.setReadPosition(p3); c1.markDelete(p2); + + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7 * 3); assertEquals(cacheManager.getSize(), entryCache.getSize()); + c1.setReadPosition(p4); c1.markDelete(p3); + + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7 * 2); assertEquals(cacheManager.getSize(), entryCache.getSize()); + ledger.deactivateCursor(c1); + + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7 * 2); assertEquals(cacheManager.getSize(), entryCache.getSize()); + c2.markDelete(p4); + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 7); assertEquals(cacheManager.getSize(), entryCache.getSize()); + ledger.deactivateCursor(c2); + factory.waitForPendingCacheEvictions(); assertEquals(entryCache.getSize(), 0); assertEquals(cacheManager.getSize(), entryCache.getSize()); } @@ -2096,6 +2129,7 @@ public void addEntryWithOffset() throws Exception { @Test public void totalSizeTest() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); + initManagedLedgerConfig(conf); conf.setMaxEntriesPerLedger(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", conf); ManagedCursor c1 = ledger.openCursor("c1"); @@ -2117,6 +2151,7 @@ public void totalSizeTest() throws Exception { @Test public void testMinimumRolloverTime() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); + initManagedLedgerConfig(conf); conf.setMaxEntriesPerLedger(1); conf.setMinimumRolloverTime(1, TimeUnit.SECONDS); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", conf); @@ -2138,6 +2173,7 @@ public void testMinimumRolloverTime() throws Exception { @Test public void testMaximumRolloverTime() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); + initManagedLedgerConfig(conf); conf.setMaxEntriesPerLedger(5); conf.setMinimumRolloverTime(1, TimeUnit.SECONDS); conf.setMaximumRolloverTime(1, TimeUnit.SECONDS); @@ -2160,6 +2196,7 @@ public void testMaximumRolloverTime() throws Exception { @Test public void testNoRolloverIfNoMetadataSession() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); + initManagedLedgerConfig(conf); conf.setMaxEntriesPerLedger(1); conf.setMinimumRolloverTime(0, TimeUnit.SECONDS); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testNoRolloverIfNoMetadataSession", conf); @@ -2188,6 +2225,7 @@ public void testNoRolloverIfNoMetadataSession() throws Exception { @Test public void testNoRolloverIfNoMetadataSessionWithExistingData() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); + initManagedLedgerConfig(conf); conf.setMaxEntriesPerLedger(2); conf.setMinimumRolloverTime(0, TimeUnit.SECONDS); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testNoRolloverIfNoMetadataSession", conf); @@ -2219,6 +2257,7 @@ public void testRetention() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionSizeInMB(10); config.setMaxEntriesPerLedger(1); config.setRetentionTime(1, TimeUnit.HOURS); @@ -2244,6 +2283,7 @@ public void testNoRetention() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionSizeInMB(0); config.setMaxEntriesPerLedger(1); // Default is no-retention @@ -2272,6 +2312,7 @@ public void testDeletionAfterRetention() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionSizeInMB(0); config.setMaxEntriesPerLedger(1); config.setRetentionTime(1, TimeUnit.SECONDS); @@ -2301,6 +2342,7 @@ public void testDeletionAfterLedgerClosedAndRetention() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionSizeInMB(0); config.setMaxEntriesPerLedger(1); config.setRetentionTime(1, TimeUnit.SECONDS); @@ -2337,6 +2379,7 @@ public void testRetention0WithEmptyLedger() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionTime(0, TimeUnit.MINUTES); config.setMaxEntriesPerLedger(1); @@ -2367,6 +2410,7 @@ public void testRetention0WithEmptyLedgerWithoutCursors() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionTime(0, TimeUnit.MINUTES); config.setMaxEntriesPerLedger(1); @@ -2389,6 +2433,7 @@ public void testInfiniteRetention() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionSizeInMB(-1); config.setRetentionTime(-1, TimeUnit.HOURS); config.setMaxEntriesPerLedger(1); @@ -2423,6 +2468,7 @@ public void testRetentionSize() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionSizeInMB(retentionSizeInMB); config.setMaxEntriesPerLedger(1); config.setRetentionTime(1, TimeUnit.HOURS); @@ -2455,6 +2501,7 @@ public void testRetentionSize() throws Exception { @Test public void testTimestampOnWorkingLedger() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); + initManagedLedgerConfig(conf); conf.setMaxEntriesPerLedger(1); conf.setRetentionSizeInMB(10); conf.setRetentionTime(1, TimeUnit.HOURS); @@ -2499,6 +2546,7 @@ public void testBackwardCompatiblityForMeta() throws Exception { final Stat[] versions = new Stat[1]; ManagedLedgerConfig conf = new ManagedLedgerConfig(); + initManagedLedgerConfig(conf); conf.setMaxEntriesPerLedger(1); conf.setRetentionSizeInMB(10); conf.setRetentionTime(1, TimeUnit.HOURS); @@ -2567,6 +2615,7 @@ public void operationFailed(MetaStoreException e) { @Test public void testGetPositionAfterN() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + initManagedLedgerConfig(managedLedgerConfig); managedLedgerConfig.setMaxEntriesPerLedger(5); ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) factory.open("testGetPositionAfterN", managedLedgerConfig); @@ -2637,6 +2686,7 @@ public void testGetPositionAfterN() throws Exception { @Test public void testGetNumberOfEntriesInStorage() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + initManagedLedgerConfig(managedLedgerConfig); managedLedgerConfig.setMaxEntriesPerLedger(5); ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) factory.open("testGetNumberOfEntriesInStorage", managedLedgerConfig); @@ -2695,6 +2745,7 @@ public void testEstimatedBacklogSize() throws Exception { @Test public void testGetNextValidPosition() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); + initManagedLedgerConfig(conf); conf.setMaxEntriesPerLedger(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testGetNextValidPosition", conf); ManagedCursor c1 = ledger.openCursor("c1"); @@ -2733,9 +2784,12 @@ public void testGetNextValidPosition() throws Exception { public void testActiveDeactiveCursorWithDiscardEntriesFromCache() throws Exception { ManagedLedgerFactoryConfig conf = new ManagedLedgerFactoryConfig(); conf.setCacheEvictionIntervalMs(10000); + conf.setCacheEvictionTimeThresholdMillis(10000); + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + initManagedLedgerConfig(managedLedgerConfig); @Cleanup("shutdown") - ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, conf); + ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, conf, managedLedgerConfig); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("cache_eviction_ledger"); // Open Cursor also adds cursor into activeCursor-container @@ -2773,8 +2827,7 @@ public void testActiveDeactiveCursorWithDiscardEntriesFromCache() throws Excepti entry.release(); } - // read after a second: as RateLimiter limits triggering of removing cache - Thread.sleep(1000); + factory.waitForPendingCacheEvictions(); List entries2 = cursor2.readEntries(readEntries); // Acknowledge only on last entry @@ -2784,6 +2837,8 @@ public void testActiveDeactiveCursorWithDiscardEntriesFromCache() throws Excepti entry.release(); } + factory.waitForPendingCacheEvictions(); + // (3) Validate: cache should remove all entries read by both active cursors log.info("expected, found : {}, {}", 5 * (totalInsertedEntries - readEntries), entryCache.getSize()); assertEquals(entryCache.getSize(), 5 * (totalInsertedEntries - readEntries)); @@ -2798,6 +2853,8 @@ public void testActiveDeactiveCursorWithDiscardEntriesFromCache() throws Excepti entry.release(); } + factory.waitForPendingCacheEvictions(); + // (4) Validate: cursor2 is active cursor and has not read these entries yet: so, cache should not remove these // entries assertEquals(entryCache.getSize(), 5 * (totalInsertedEntries - readEntries)); @@ -2805,6 +2862,8 @@ public void testActiveDeactiveCursorWithDiscardEntriesFromCache() throws Excepti ledger.deactivateCursor(cursor1); ledger.deactivateCursor(cursor2); + factory.waitForPendingCacheEvictions(); + // (5) Validate: cursor2 is not active cursor now: cache should have removed all entries read by active cursor1 assertEquals(entryCache.getSize(), 0); @@ -2894,6 +2953,7 @@ public void testLazyRecoverCursor() throws Exception { }, 10, TimeUnit.SECONDS); ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + initManagedLedgerConfig(managedLedgerConfig); managedLedgerConfig.setLazyCursorRecovery(true); Long startLedgerRecovery = System.currentTimeMillis(); @@ -3046,7 +3106,7 @@ public void openCursorComplete(ManagedCursor cursor, Object ctx) { @Test public void testConsumerSubscriptionInitializePosition() throws Exception{ final int MAX_ENTRY_PER_LEDGER = 2; - ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(MAX_ENTRY_PER_LEDGER); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(MAX_ENTRY_PER_LEDGER); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("lastest_earliest_ledger", config); final int totalInsertedEntries = 20; @@ -3080,14 +3140,14 @@ public void testConsumerSubscriptionInitializePosition() throws Exception{ @Test public void testManagedLedgerAutoCreate() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setCreateIfMissing(true); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setCreateIfMissing(true); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("test", config); assertNotNull(ledger); } @Test public void testManagedLedgerWithoutAutoCreate() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setCreateIfMissing(false); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setCreateIfMissing(false); try { factory.open("testManagedLedgerWithoutAutoCreate", config); @@ -3101,7 +3161,7 @@ public void testManagedLedgerWithoutAutoCreate() throws Exception { @Test public void testManagedLedgerWithCreateLedgerTimeOut() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setMetadataOperationsTimeoutSeconds(3); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setMetadataOperationsTimeoutSeconds(3); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("timeout_ledger_test", config); BookKeeper bk = mock(BookKeeper.class); @@ -3131,7 +3191,7 @@ public void testManagedLedgerWithCreateLedgerTimeOut() throws Exception { */ @Test public void testManagedLedgerWithReadEntryTimeOut() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setReadEntryTimeoutSeconds(1); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setReadEntryTimeoutSeconds(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("timeout_ledger_test", config); Position position = ledger.addEntry("entry-1".getBytes()); @@ -3202,7 +3262,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { @Test public void testAddEntryResponseTimeout() throws Exception { // Create ML with feature Add Entry Timeout Check. - final ManagedLedgerConfig config = new ManagedLedgerConfig().setAddEntryTimeoutSeconds(2); + final ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setAddEntryTimeoutSeconds(2); final ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("ml1", config); final ManagedCursor cursor = ledger.openCursor("c1"); final CollectCtxAddEntryCallback collectCtxAddEntryCallback = new CollectCtxAddEntryCallback(); @@ -3257,7 +3317,7 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { */ @Test(timeOut = 20000) public void testManagedLedgerWithAddEntryTimeOut() throws Exception { - ManagedLedgerConfig config = new ManagedLedgerConfig().setAddEntryTimeoutSeconds(1); + ManagedLedgerConfig config = initManagedLedgerConfig(new ManagedLedgerConfig()).setAddEntryTimeoutSeconds(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("timeout_ledger_test", config); BookKeeper bk = mock(BookKeeper.class); @@ -3460,7 +3520,7 @@ public void operationFailed(MetaStoreException e) { private void createLedger(ManagedLedgerFactoryImpl factory, MutableObject ledger1, MutableObject cursor1, boolean checkOwnershipFlag) throws Exception { CountDownLatch latch = new CountDownLatch(1); - factory.asyncOpen("my_test_ledger", new ManagedLedgerConfig(), new OpenLedgerCallback() { + factory.asyncOpen("my_test_ledger", initManagedLedgerConfig(new ManagedLedgerConfig()), new OpenLedgerCallback() { @Override public void openLedgerComplete(ManagedLedger ledger, Object ctx) { ledger1.setValue(ledger); @@ -3495,7 +3555,7 @@ public void deleteWithoutOpen() throws Exception { factory.delete("my_test_ledger"); try { - factory.open("my_test_ledger", new ManagedLedgerConfig().setCreateIfMissing(false)); + factory.open("my_test_ledger", initManagedLedgerConfig(new ManagedLedgerConfig()).setCreateIfMissing(false)); fail("Should have failed"); } catch (ManagedLedgerNotFoundException e) { // Expected @@ -3507,6 +3567,7 @@ private abstract class MockedPlacementPolicy implements EnsemblePlacementPolicy{ @Test(timeOut = 10000) public void testManagedLedgerWithPlacementPolicyInCustomMetadata() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + initManagedLedgerConfig(managedLedgerConfig); managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyClassName(MockedPlacementPolicy.class); managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(Collections.singletonMap("key", "value")); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", managedLedgerConfig); @@ -3538,6 +3599,7 @@ public static void retryStrategically(Predicate predicate, int retryCount, @Test public void testManagedLedgerRollOverIfFull() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionTime(1, TimeUnit.SECONDS); config.setMaxEntriesPerLedger(2); config.setMaximumRolloverTime(500, TimeUnit.MILLISECONDS); @@ -3577,6 +3639,7 @@ public void testManagedLedgerRollOverIfFull() throws Exception { @Test public void testLedgerReachMaximumRolloverTime() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMinimumRolloverTime(1, TimeUnit.MILLISECONDS); config.setMaximumRolloverTime(1, TimeUnit.SECONDS); @@ -3592,6 +3655,7 @@ public void testLedgerReachMaximumRolloverTime() throws Exception { @Test(groups = "flaky") public void testLedgerNotRolloverWithoutOpenState() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMaxEntriesPerLedger(2); ManagedLedgerImpl ml = spy((ManagedLedgerImpl)factory.open("ledger-not-rollover-without-open-state", config)); @@ -3612,6 +3676,7 @@ public void testLedgerNotRolloverWithoutOpenState() throws Exception { @Test public void testExpiredLedgerDeletionAfterManagedLedgerRestart() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionTime(1, TimeUnit.SECONDS); config.setMaxEntriesPerLedger(2); config.setMinimumRolloverTime(1, TimeUnit.MILLISECONDS); @@ -3655,6 +3720,7 @@ public void testExpiredLedgerDeletionAfterManagedLedgerRestart() throws Exceptio @Test(timeOut = 20000) public void testAsyncTruncateLedgerRetention() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setRetentionSizeInMB(50); config.setRetentionTime(1, TimeUnit.DAYS); @@ -3680,6 +3746,7 @@ public void testAsyncTruncateLedgerRetention() throws Exception { @Test(timeOut = 20000) public void testAsyncTruncateLedgerSlowestCursor() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("truncate_ledger", config); ManagedCursor cursor = ledger.openCursor("test-cursor"); @@ -3723,6 +3790,7 @@ public void testOpEntryAdd_toString_doesNotThrowNPE(){ @Test public void testInvalidateReadHandleWhenDeleteLedger() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMaxEntriesPerLedger(1); // Verify the read handle should be invalidated after ledger been removed. @@ -3761,6 +3829,7 @@ public void testInvalidateReadHandleWhenDeleteLedger() throws Exception { @Test public void testLockReleaseWhenTrimLedger() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMaxEntriesPerLedger(1); ManagedLedgerImpl ledger = spy((ManagedLedgerImpl)factory.open("testLockReleaseWhenTrimLedger", config)); @@ -3793,6 +3862,7 @@ public void testLockReleaseWhenTrimLedger() throws Exception { @Test public void testInvalidateReadHandleWhenConsumed() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMaxEntriesPerLedger(1); // Verify the read handle should be invalidated when all cursors consumed // even if the ledger can not been removed due to the data retention @@ -3850,6 +3920,7 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception { @Test public void testDoNotGetOffloadPoliciesMultipleTimesWhenTrimLedgers() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMaxEntriesPerLedger(1); config.setMaxSizePerLedgerMb(1); LedgerOffloader ledgerOffloader = mock(NullLedgerOffloader.class); @@ -3970,6 +4041,7 @@ public void testInactiveLedgerRollOver() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setInactiveLedgerRollOverTime(inactiveLedgerRollOverTimeMs, TimeUnit.MILLISECONDS); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("rollover_inactive", config); ManagedCursor cursor = ledger.openCursor("c1"); @@ -4008,6 +4080,7 @@ public void testDontRollOverEmptyInactiveLedgers() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setInactiveLedgerRollOverTime(inactiveLedgerRollOverTimeMs, TimeUnit.MILLISECONDS); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("rollover_inactive", config); ManagedCursor cursor = ledger.openCursor("c1"); @@ -4033,6 +4106,7 @@ public void testDontRollOverInactiveLedgersWhenMetadataServiceInvalid() throws E // mock metadata service invalid when(factory.isMetadataServiceAvailable()).thenReturn(false); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setInactiveLedgerRollOverTime(inactiveLedgerRollOverTimeMs, TimeUnit.MILLISECONDS); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("rollover_inactive", config); @@ -4054,6 +4128,7 @@ public void testOffloadTaskCancelled() throws Exception { @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMaxEntriesPerLedger(2); config.setMinimumRolloverTime(0, TimeUnit.SECONDS); @@ -4091,7 +4166,7 @@ public void testOffloadTaskCancelled() throws Exception { @Test public void testGetTheSlowestNonDurationReadPosition() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("test_", - new ManagedLedgerConfig().setMaxEntriesPerLedger(1).setRetentionTime(-1, TimeUnit.SECONDS) + initManagedLedgerConfig(new ManagedLedgerConfig()).setMaxEntriesPerLedger(1).setRetentionTime(-1, TimeUnit.SECONDS) .setRetentionSizeInMB(-1)); ledger.openCursor("c1"); @@ -4197,6 +4272,7 @@ public void testIsNoMessagesAfterPos() throws Exception { @Test public void testGetEstimatedBacklogSize() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMaxEntriesPerLedger(2); config.setRetentionTime(-1, TimeUnit.SECONDS); config.setRetentionSizeInMB(-1); @@ -4243,6 +4319,7 @@ public void testNonDurableCursorCreateForInactiveLedger() throws Exception { BookKeeper spyBookKeeper = spy(bkc); ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, spyBookKeeper); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setInactiveLedgerRollOverTime(10, TimeUnit.MILLISECONDS); ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open(mlName, config); ml.addEntry("entry".getBytes(UTF_8)); @@ -4295,6 +4372,7 @@ public void testNoOrphanScheduledTasksAfterCloseML() throws Exception { String mlName = UUID.randomUUID().toString(); ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMetadataOperationsTimeoutSeconds(3600); // Calculate pending task count. @@ -4395,6 +4473,7 @@ public Object[][] closeLedgerByAddEntry() { public void testDeleteCurrentLedgerWhenItIsClosed(boolean closeLedgerByAddEntry) throws Exception { // Setup: Open a manageLedger with one initial entry. ManagedLedgerConfig config = new ManagedLedgerConfig(); + initManagedLedgerConfig(config); config.setMaxEntriesPerLedger(10); ManagedLedgerImpl ml = spy((ManagedLedgerImpl) factory.open("testDeleteCurrentLedgerWhenItIsClosed", config)); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheTest.java new file mode 100644 index 0000000000000..e9dfad0e93414 --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheTest.java @@ -0,0 +1,399 @@ +/* + * 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.cache; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNotSame; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import io.netty.buffer.Unpooled; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import lombok.Cleanup; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.util.Reflections; +import org.assertj.core.groups.Tuple; +import org.awaitility.Awaitility; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +public class RangeCacheTest { + + @Test + public void simple() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + + putToCache(cache, 0, "0"); + putToCache(cache, 1, "1"); + + assertEquals(cache.getSize(), 2); + assertEquals(cache.getNumberOfEntries(), 2); + + CachedEntry s = cache.get(createPosition(0)); + assertEquals(s.getData(), "0".getBytes()); + assertEquals(s.refCnt(), 2); + s.release(); + + CachedEntry s1 = cache.get(createPosition(0)); + CachedEntry s2 = cache.get(createPosition(0)); + assertEquals(s1, s2); + assertEquals(s1.refCnt(), 3); + s1.release(); + s2.release(); + + assertNull(cache.get(createPosition(2))); + + putToCache(cache, 2, "2"); + putToCache(cache, 8, "8"); + putToCache(cache, 11, "11"); + + assertEquals(cache.getSize(), 6); + assertEquals(cache.getNumberOfEntries(), 5); + + cache.removeRange(createPosition(1), createPosition(5), true); + assertEquals(cache.getSize(), 4); + assertEquals(cache.getNumberOfEntries(), 3); + + cache.removeRange(createPosition(2), createPosition(8), false); + assertEquals(cache.getSize(), 4); + assertEquals(cache.getNumberOfEntries(), 3); + + cache.removeRange(createPosition(0), createPosition(100), false); + assertEquals(cache.getSize(), 0); + assertEquals(cache.getNumberOfEntries(), 0); + + cache.removeRange(createPosition(0), createPosition(100), false); + assertEquals(cache.getSize(), 0); + assertEquals(cache.getNumberOfEntries(), 0); + } + + private void putToCache(RangeCache cache, int i, String str) { + Position position = createPosition(i); + CachedEntry cachedEntry = createCachedEntry(position, str); + cache.put(position, cachedEntry); + } + + private static CachedEntry createCachedEntry(int i, String str) { + return createCachedEntry(createPosition(i), str); + } + + private static CachedEntry createCachedEntry(Position position, String str) { + return CachedEntryImpl.create(position, Unpooled.wrappedBuffer(str.getBytes())); + } + + private static Position createPosition(int i) { + return PositionFactory.create(0, i); + } + + @DataProvider + public static Object[][] retainBeforeEviction() { + return new Object[][]{ { true }, { false } }; + } + + + @Test(dataProvider = "retainBeforeEviction") + public void customTimeExtraction(boolean retain) { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + + putToCache(cache, 1, "1"); + putToCache(cache, 22, "22"); + putToCache(cache, 333, "333"); + long timestamp = System.nanoTime(); + putToCache(cache, 4444, "4444"); + + assertEquals(cache.getSize(), 10); + assertEquals(cache.getNumberOfEntries(), 4); + final var retainedEntries = cache.getRange(createPosition(1), createPosition(4444)); + for (final var entry : retainedEntries) { + assertEquals(entry.refCnt(), 2); + if (!retain) { + entry.release(); + } + } + + Pair evictedSize = removalQueue.evictLEntriesBeforeTimestamp(timestamp); + assertEquals(evictedSize.getRight().longValue(), 6); + assertEquals(evictedSize.getLeft().longValue(), 3); + assertEquals(cache.getSize(), 4); + assertEquals(cache.getNumberOfEntries(), 1); + + if (retain) { + final var valueToRefCnt = + retainedEntries.stream().collect(Collectors.toMap(cachedEntry -> new String(cachedEntry.getData()), + cachedEntry -> cachedEntry.refCnt())); + assertEquals(valueToRefCnt, Map.of("1", 1, "22", 1, "333", 1, "4444", 2)); + retainedEntries.forEach(Entry::release); + } else { + final var valueToRefCnt = retainedEntries.stream().filter(v -> v.refCnt() > 0).collect(Collectors.toMap( + cachedEntry -> new String(cachedEntry.getData()), CachedEntry::refCnt)); + assertEquals(valueToRefCnt, Map.of("4444", 1)); + } + } + + @Test + public void doubleInsert() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + + CachedEntry s0 = createCachedEntry(0, "zero"); + assertEquals(s0.refCnt(), 1); + assertTrue(cache.put(s0.getPosition(), s0)); + assertEquals(s0.refCnt(), 1); + + CachedEntry one = createCachedEntry(1, "one"); + assertTrue(cache.put(one.getPosition(), one)); + assertEquals(createPosition(1), one.getPosition()); + + assertEquals(cache.getSize(), 7); + assertEquals(cache.getNumberOfEntries(), 2); + CachedEntry s = cache.get(createPosition(1)); + assertEquals(s.getData(), "one".getBytes()); + assertEquals(s.refCnt(), 2); + + CachedEntry s1 = createCachedEntry(1, "uno"); + assertEquals(s1.refCnt(), 1); + assertFalse(cache.put(s1.getPosition(), s1)); + assertEquals(s1.refCnt(), 1); + s1.release(); + + // Should not have been overridden in cache + assertEquals(cache.getSize(), 7); + assertEquals(cache.getNumberOfEntries(), 2); + assertEquals(cache.get(createPosition(1)).getData(), "one".getBytes()); + } + + @Test + public void getRange() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + + putToCache(cache, 0, "0"); + putToCache(cache, 1, "1"); + putToCache(cache, 3, "3"); + putToCache(cache, 5, "5"); + + assertThat(cache.getRange(createPosition(1), createPosition(8))) + .map(entry -> Tuple.tuple(entry.getPosition(), new String(entry.getData()))) + .containsExactly( + Tuple.tuple(createPosition(1), "1"), + Tuple.tuple(createPosition(3), "3"), + Tuple.tuple(createPosition(5), "5") + ); + + putToCache(cache, 8, "8"); + + assertThat(cache.getRange(createPosition(1), createPosition(8))) + .map(entry -> Tuple.tuple(entry.getPosition(), new String(entry.getData()))) + .containsExactly( + Tuple.tuple(createPosition(1), "1"), + Tuple.tuple(createPosition(3), "3"), + Tuple.tuple(createPosition(5), "5"), + Tuple.tuple(createPosition(8), "8") + ); + + cache.clear(); + assertEquals(cache.getSize(), 0); + assertEquals(cache.getNumberOfEntries(), 0); + } + + @Test + public void eviction() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + + putToCache(cache, 0, "zero"); + putToCache(cache, 1, "one"); + putToCache(cache, 2, "two"); + putToCache(cache, 3, "three"); + + // This should remove the LRU entries: 0, 1 whose combined size is 7 + assertEquals(removalQueue.evictLeastAccessedEntries(5), Pair.of(2, (long) 7)); + + assertEquals(cache.getNumberOfEntries(), 2); + assertEquals(cache.getSize(), 8); + assertNull(cache.get(createPosition(0))); + assertNull(cache.get(createPosition(1))); + assertEquals(cache.get(createPosition(2)).getData(), "two".getBytes()); + assertEquals(cache.get(createPosition(3)).getData(), "three".getBytes()); + + assertEquals(removalQueue.evictLeastAccessedEntries(100), Pair.of(2, (long) 8)); + assertEquals(cache.getNumberOfEntries(), 0); + assertEquals(cache.getSize(), 0); + assertNull(cache.get(createPosition(0))); + assertNull(cache.get(createPosition(1))); + assertNull(cache.get(createPosition(2))); + assertNull(cache.get(createPosition(3))); + + try { + removalQueue.evictLeastAccessedEntries(0); + fail("should throw exception"); + } catch (IllegalArgumentException e) { + // ok + } + + try { + removalQueue.evictLeastAccessedEntries(-1); + fail("should throw exception"); + } catch (IllegalArgumentException e) { + // ok + } + } + + @Test + public void evictions() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + + int expectedSize = 0; + for (int i = 0; i < 100; i++) { + String string = Integer.toString(i); + expectedSize += string.length(); + putToCache(cache, i, string); + } + + assertEquals(cache.getSize(), expectedSize); + Pair res = removalQueue.evictLeastAccessedEntries(1); + assertEquals((int) res.getLeft(), 1); + assertEquals((long) res.getRight(), 1); + expectedSize -= 1; + assertEquals(cache.getSize(), expectedSize); + + res = removalQueue.evictLeastAccessedEntries(10); + assertEquals((int) res.getLeft(), 10); + assertEquals((long) res.getRight(), 11); + expectedSize -= 11; + assertEquals(cache.getSize(), expectedSize); + + res = removalQueue.evictLeastAccessedEntries(expectedSize); + assertEquals((int) res.getLeft(), 89); + assertEquals((long) res.getRight(), expectedSize); + assertEquals(cache.getSize(), 0); + + expectedSize = 0; + for (int i = 0; i < 100; i++) { + String string = Integer.toString(i); + expectedSize += string.length(); + putToCache(cache, i, string); + } + + assertEquals(cache.getSize(), expectedSize); + + res = cache.removeRange(createPosition(10), createPosition(20), false); + assertEquals((int) res.getLeft(), 10); + assertEquals((long) res.getRight(), 20); + expectedSize -= 20; + assertEquals(cache.getSize(), expectedSize); + } + + @Test + public void testPutWhileClearIsCalledConcurrently() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + int numberOfThreads = 8; + @Cleanup("shutdownNow") + ScheduledExecutorService executor = Executors.newScheduledThreadPool(numberOfThreads); + for (int i = 0; i < numberOfThreads; i++) { + executor.scheduleWithFixedDelay(cache::clear, 0, 1, TimeUnit.MILLISECONDS); + } + for (int i = 0; i < 200000; i++) { + putToCache(cache, i, Integer.toString(i)); + } + executor.shutdown(); + // ensure that no clear operation got into endless loop + Awaitility.await().untilAsserted(() -> assertTrue(executor.isTerminated())); + // ensure that clear can be called and all entries are removed + cache.clear(); + assertEquals(cache.getNumberOfEntries(), 0); + } + + @Test + public void testPutSameObj() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + CachedEntry s0 = createCachedEntry(0, "zero"); + assertEquals(s0.refCnt(), 1); + assertTrue(cache.put(s0.getPosition(), s0)); + assertFalse(cache.put(s0.getPosition(), s0)); + } + + @Test + public void testRemoveEntryWithInvalidRefCount() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + CachedEntry value = createCachedEntry(1, "1"); + cache.put(value.getPosition(), value); + // release the value to make the reference count invalid + value.release(); + cache.clear(); + assertEquals(cache.getNumberOfEntries(), 0); + } + + @Test + public void testInvalidMatchingKey() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + CachedEntry value = createCachedEntry(1, "1"); + cache.put(value.getPosition(), value); + assertNotNull(cache.get(value.getPosition())); + // change the entryId to make the entry invalid for the cache + Reflections.getAllFields(value.getClass()).stream() + .filter(field -> field.getName().equals("entryId")) + .forEach(field -> { + field.setAccessible(true); + try { + field.set(value, 123); + } catch (IllegalAccessException e) { + fail("Failed to set matching key"); + } + }); + assertNull(cache.get(value.getPosition())); + cache.clear(); + assertEquals(cache.getNumberOfEntries(), 0); + } + + @Test + public void testGetKeyWithDifferentInstance() { + RangeCacheRemovalQueue removalQueue = new RangeCacheRemovalQueue(); + RangeCache cache = new RangeCache(removalQueue); + Position key = createPosition(129); + CachedEntry value = createCachedEntry(key, "129"); + cache.put(key, value); + // create a different instance of the key + Position key2 = createPosition(129); + // key and key2 are different instances but they are equal + assertNotSame(key, key2); + assertEquals(key, key2); + // get the value using key2 + CachedEntry value2 = cache.get(key2); + // the value should be found + assertEquals(value2.getData(), "129".getBytes()); + } +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java deleted file mode 100644 index b6914fd8efe49..0000000000000 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java +++ /dev/null @@ -1,383 +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.util; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotSame; -import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; -import com.google.common.collect.Lists; -import io.netty.util.AbstractReferenceCounted; -import io.netty.util.ReferenceCounted; -import java.util.Map; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import lombok.Cleanup; -import lombok.Data; -import org.apache.commons.lang3.tuple.Pair; -import org.awaitility.Awaitility; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; - -public class RangeCacheTest { - - @Data - class RefString extends AbstractReferenceCounted implements RangeCache.ValueWithKeyValidation { - String s; - Integer matchingKey; - - RefString(String s) { - this(s, null); - } - - RefString(String s, Integer matchingKey) { - super(); - this.s = s; - this.matchingKey = matchingKey != null ? matchingKey : Integer.parseInt(s); - setRefCnt(1); - } - - @Override - protected void deallocate() { - s = null; - } - - @Override - public ReferenceCounted touch(Object hint) { - return this; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof RefString) { - return this.s.equals(((RefString) obj).s); - } else if (obj instanceof String) { - return this.s.equals((String) obj); - } - - return false; - } - - @Override - public boolean matchesKey(Integer key) { - return matchingKey.equals(key); - } - } - - @Test - public void simple() { - RangeCache cache = new RangeCache<>(); - - cache.put(0, new RefString("0")); - cache.put(1, new RefString("1")); - - assertEquals(cache.getSize(), 2); - assertEquals(cache.getNumberOfEntries(), 2); - - RefString s = cache.get(0); - assertEquals(s.s, "0"); - assertEquals(s.refCnt(), 2); - s.release(); - - RefString s1 = cache.get(0); - RefString s2 = cache.get(0); - assertEquals(s1, s2); - assertEquals(s1.refCnt(), 3); - s1.release(); - s2.release(); - - assertNull(cache.get(2)); - - cache.put(2, new RefString("2")); - cache.put(8, new RefString("8")); - cache.put(11, new RefString("11")); - - assertEquals(cache.getSize(), 5); - assertEquals(cache.getNumberOfEntries(), 5); - - cache.removeRange(1, 5, true); - assertEquals(cache.getSize(), 3); - assertEquals(cache.getNumberOfEntries(), 3); - - cache.removeRange(2, 8, false); - assertEquals(cache.getSize(), 3); - assertEquals(cache.getNumberOfEntries(), 3); - - cache.removeRange(0, 100, false); - assertEquals(cache.getSize(), 0); - assertEquals(cache.getNumberOfEntries(), 0); - - cache.removeRange(0, 100, false); - assertEquals(cache.getSize(), 0); - assertEquals(cache.getNumberOfEntries(), 0); - } - - @Test - public void customWeighter() { - RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - - cache.put(0, new RefString("zero", 0)); - cache.put(1, new RefString("one", 1)); - - assertEquals(cache.getSize(), 7); - assertEquals(cache.getNumberOfEntries(), 2); - } - - @DataProvider - public static Object[][] retainBeforeEviction() { - return new Object[][]{ { true }, { false } }; - } - - - @Test(dataProvider = "retainBeforeEviction") - public void customTimeExtraction(boolean retain) { - RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> x.s.length()); - - cache.put(1, new RefString("1")); - cache.put(22, new RefString("22")); - cache.put(333, new RefString("333")); - cache.put(4444, new RefString("4444")); - - assertEquals(cache.getSize(), 10); - assertEquals(cache.getNumberOfEntries(), 4); - final var retainedEntries = cache.getRange(1, 4444); - for (final var entry : retainedEntries) { - assertEquals(entry.refCnt(), 2); - if (!retain) { - entry.release(); - } - } - - Pair evictedSize = cache.evictLEntriesBeforeTimestamp(3); - assertEquals(evictedSize.getRight().longValue(), 6); - assertEquals(evictedSize.getLeft().longValue(), 3); - assertEquals(cache.getSize(), 4); - assertEquals(cache.getNumberOfEntries(), 1); - - if (retain) { - final var valueToRefCnt = retainedEntries.stream().collect(Collectors.toMap(RefString::getS, - AbstractReferenceCounted::refCnt)); - assertEquals(valueToRefCnt, Map.of("1", 1, "22", 1, "333", 1, "4444", 2)); - retainedEntries.forEach(AbstractReferenceCounted::release); - } else { - final var valueToRefCnt = retainedEntries.stream().filter(v -> v.refCnt() > 0).collect(Collectors.toMap( - RefString::getS, AbstractReferenceCounted::refCnt)); - assertEquals(valueToRefCnt, Map.of("4444", 1)); - } - } - - @Test - public void doubleInsert() { - RangeCache cache = new RangeCache<>(); - - RefString s0 = new RefString("zero", 0); - assertEquals(s0.refCnt(), 1); - assertTrue(cache.put(0, s0)); - assertEquals(s0.refCnt(), 1); - - cache.put(1, new RefString("one", 1)); - - assertEquals(cache.getSize(), 2); - assertEquals(cache.getNumberOfEntries(), 2); - RefString s = cache.get(1); - assertEquals(s.s, "one"); - assertEquals(s.refCnt(), 2); - - RefString s1 = new RefString("uno", 1); - assertEquals(s1.refCnt(), 1); - assertFalse(cache.put(1, s1)); - assertEquals(s1.refCnt(), 1); - s1.release(); - - // Should not have been overridden in cache - assertEquals(cache.getSize(), 2); - assertEquals(cache.getNumberOfEntries(), 2); - assertEquals(cache.get(1).s, "one"); - } - - @Test - public void getRange() { - RangeCache cache = new RangeCache<>(); - - cache.put(0, new RefString("0")); - cache.put(1, new RefString("1")); - cache.put(3, new RefString("3")); - cache.put(5, new RefString("5")); - - assertEquals(cache.getRange(1, 8), - Lists.newArrayList(new RefString("1"), new RefString("3"), new RefString("5"))); - - cache.put(8, new RefString("8")); - assertEquals(cache.getRange(1, 8), - Lists.newArrayList(new RefString("1"), new RefString("3"), new RefString("5"), new RefString("8"))); - - cache.clear(); - assertEquals(cache.getSize(), 0); - assertEquals(cache.getNumberOfEntries(), 0); - } - - @Test - public void eviction() { - RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - - cache.put(0, new RefString("zero", 0)); - cache.put(1, new RefString("one", 1)); - cache.put(2, new RefString("two", 2)); - cache.put(3, new RefString("three", 3)); - - // This should remove the LRU entries: 0, 1 whose combined size is 7 - assertEquals(cache.evictLeastAccessedEntries(5), Pair.of(2, (long) 7)); - - assertEquals(cache.getNumberOfEntries(), 2); - assertEquals(cache.getSize(), 8); - assertNull(cache.get(0)); - assertNull(cache.get(1)); - assertEquals(cache.get(2).s, "two"); - assertEquals(cache.get(3).s, "three"); - - assertEquals(cache.evictLeastAccessedEntries(100), Pair.of(2, (long) 8)); - assertEquals(cache.getNumberOfEntries(), 0); - assertEquals(cache.getSize(), 0); - assertNull(cache.get(0)); - assertNull(cache.get(1)); - assertNull(cache.get(2)); - assertNull(cache.get(3)); - - try { - cache.evictLeastAccessedEntries(0); - fail("should throw exception"); - } catch (IllegalArgumentException e) { - // ok - } - - try { - cache.evictLeastAccessedEntries(-1); - fail("should throw exception"); - } catch (IllegalArgumentException e) { - // ok - } - } - - @Test - public void evictions() { - RangeCache cache = new RangeCache<>(); - - for (int i = 0; i < 100; i++) { - cache.put(i, new RefString(Integer.toString(i))); - } - - assertEquals(cache.getSize(), 100); - Pair res = cache.evictLeastAccessedEntries(1); - assertEquals((int) res.getLeft(), 1); - assertEquals((long) res.getRight(), 1); - assertEquals(cache.getSize(), 99); - - res = cache.evictLeastAccessedEntries(10); - assertEquals((int) res.getLeft(), 10); - assertEquals((long) res.getRight(), 10); - assertEquals(cache.getSize(), 89); - - res = cache.evictLeastAccessedEntries(100); - assertEquals((int) res.getLeft(), 89); - assertEquals((long) res.getRight(), 89); - assertEquals(cache.getSize(), 0); - - for (int i = 0; i < 100; i++) { - cache.put(i, new RefString(Integer.toString(i))); - } - - assertEquals(cache.getSize(), 100); - - res = cache.removeRange(10, 20, false); - assertEquals((int) res.getLeft(), 10); - assertEquals((long) res.getRight(), 10); - assertEquals(cache.getSize(), 90); - } - - @Test - public void testPutWhileClearIsCalledConcurrently() { - RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - int numberOfThreads = 8; - @Cleanup("shutdownNow") - ScheduledExecutorService executor = Executors.newScheduledThreadPool(numberOfThreads); - for (int i = 0; i < numberOfThreads; i++) { - executor.scheduleWithFixedDelay(cache::clear, 0, 1, TimeUnit.MILLISECONDS); - } - for (int i = 0; i < 200000; i++) { - cache.put(i, new RefString(String.valueOf(i))); - } - executor.shutdown(); - // ensure that no clear operation got into endless loop - Awaitility.await().untilAsserted(() -> assertTrue(executor.isTerminated())); - // ensure that clear can be called and all entries are removed - cache.clear(); - assertEquals(cache.getNumberOfEntries(), 0); - } - - @Test - public void testPutSameObj() { - RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - RefString s0 = new RefString("zero", 0); - assertEquals(s0.refCnt(), 1); - assertTrue(cache.put(0, s0)); - assertFalse(cache.put(0, s0)); - } - - @Test - public void testRemoveEntryWithInvalidRefCount() { - RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - RefString value = new RefString("1"); - cache.put(1, value); - // release the value to make the reference count invalid - value.release(); - cache.clear(); - assertEquals(cache.getNumberOfEntries(), 0); - } - - @Test - public void testRemoveEntryWithInvalidMatchingKey() { - RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - RefString value = new RefString("1"); - cache.put(1, value); - // change the matching key to make it invalid - value.setMatchingKey(123); - cache.clear(); - assertEquals(cache.getNumberOfEntries(), 0); - } - - @Test - public void testGetKeyWithDifferentInstance() { - RangeCache cache = new RangeCache<>(); - Integer key = 129; - cache.put(key, new RefString("129")); - // create a different instance of the key - Integer key2 = Integer.valueOf(129); - // key and key2 are different instances but they are equal - assertNotSame(key, key2); - assertEquals(key, key2); - // get the value using key2 - RefString s = cache.get(key2); - // the value should be found - assertEquals(s.s, "129"); - } -} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java b/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java index c7685cfaa6594..e7f657cde81fe 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java @@ -26,6 +26,7 @@ import lombok.SneakyThrows; import org.apache.bookkeeper.client.PulsarMockBookKeeper; import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; @@ -84,11 +85,18 @@ public final void setUp(Method method) throws Exception { ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig(); initManagedLedgerFactoryConfig(managedLedgerFactoryConfig); - factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + initManagedLedgerConfig(managedLedgerConfig); + factory = + new ManagedLedgerFactoryImpl(metadataStore, bkc, managedLedgerFactoryConfig, managedLedgerConfig); setUpTestCase(); } + protected ManagedLedgerConfig initManagedLedgerConfig(ManagedLedgerConfig config) { + return config; + } + protected void initManagedLedgerFactoryConfig(ManagedLedgerFactoryConfig config) { // increase default cache eviction interval so that caching could be tested with less flakyness config.setCacheEvictionIntervalMs(200); @@ -108,14 +116,19 @@ public final void tearDown(Method method) { } try { LOG.info("@@@@@@@@@ stopping " + method); - try { - factory.shutdownAsync().get(10, TimeUnit.SECONDS); - } catch (ManagedLedgerException.ManagedLedgerFactoryClosedException e) { - // ignore + if (factory != null) { + try { + factory.shutdownAsync().get(10, TimeUnit.SECONDS); + } catch (ManagedLedgerException.ManagedLedgerFactoryClosedException e) { + // ignore + } + factory = null; } - factory = null; stopBookKeeper(); - metadataStore.close(); + if (metadataStore != null) { + metadataStore.close(); + metadataStore = null; + } LOG.info("--------- stopped {}", method); } catch (Exception e) { LOG.error("tearDown Error", e); @@ -158,7 +171,10 @@ protected void startBookKeeper() throws Exception { } protected void stopBookKeeper() { - bkc.shutdown(); + if (bkc != null) { + bkc.shutdown(); + bkc = null; + } } protected void stopMetadataStore() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index c4f1379f43789..6021c41142a5e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.broker.service.nonpersistent; -import static org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl.create; +import static org.apache.bookkeeper.mledger.impl.EntryImpl.create; import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import com.carrotsearch.hppc.ObjectObjectHashMap; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/BacklogConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/BacklogConsumerTest.java new file mode 100644 index 0000000000000..5bd2a213523d4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/BacklogConsumerTest.java @@ -0,0 +1,142 @@ +/* + * 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.pulsar.broker.cache; + +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.SubscriptionType; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Test(groups = "broker-api") +@Slf4j +public class BacklogConsumerTest extends ProducerConsumerBase { + private static final int RECEIVE_TIMEOUT_SHORT_MILLIS = 200; + + @BeforeMethod + @Override + protected void setup() throws Exception { + this.conf.setClusterName("test"); + internalSetup(); + producerBaseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + internalCleanup(); + } + + @DataProvider(name = "ackReceiptEnabled") + public Object[][] ackReceiptEnabled() { + return new Object[][] { { true }, { false } }; + } + + @Test(timeOut = 100000, dataProvider = "ackReceiptEnabled") + public void testDeactivatingBacklogConsumer(boolean ackReceiptEnabled) throws Exception { + log.info("-- Starting {} test --", methodName); + + final long batchMessageDelayMs = 100; + final int receiverSize = 10; + final String topicName = "cache-topic"; + final String topic = "persistent://my-property/my-ns/" + topicName; + final String sub1 = "faster-sub1"; + final String sub2 = "slower-sub2"; + + // 1. Subscriber Faster subscriber: let it consume all messages immediately + @Cleanup + Consumer subscriber1 = pulsarClient.newConsumer() + .topic("persistent://my-property/my-ns/" + topicName).subscriptionName(sub1) + .isAckReceiptEnabled(ackReceiptEnabled) + .subscriptionType(SubscriptionType.Shared).receiverQueueSize(receiverSize).subscribe(); + // 1.b. Subscriber Slow subscriber: + @Cleanup + Consumer subscriber2 = pulsarClient.newConsumer() + .topic("persistent://my-property/my-ns/" + topicName).subscriptionName(sub2) + .isAckReceiptEnabled(ackReceiptEnabled) + .subscriptionType(SubscriptionType.Shared).receiverQueueSize(receiverSize).subscribe(); + + ProducerBuilder producerBuilder = pulsarClient.newProducer().topic(topic); + producerBuilder.enableBatching(true).batchingMaxPublishDelay(batchMessageDelayMs, TimeUnit.MILLISECONDS) + .batchingMaxMessages(5); + @Cleanup + Producer producer = producerBuilder.create(); + + PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) topicRef.getManagedLedger(); + + // reflection to set/get cache-backlog fields value: + final long maxMessageCacheRetentionTimeMillis = conf.getManagedLedgerCacheEvictionTimeThresholdMillis(); + final long maxActiveCursorBacklogEntries = conf.getManagedLedgerCursorBackloggedThreshold(); + + Message msg; + final int totalMsgs = (int) maxActiveCursorBacklogEntries + receiverSize + 1; + // 2. Produce messages + for (int i = 0; i < totalMsgs; i++) { + String message = "my-message-" + i; + producer.send(message.getBytes()); + } + // 3. Consume messages: at Faster subscriber + for (int i = 0; i < totalMsgs; i++) { + msg = subscriber1.receive(RECEIVE_TIMEOUT_SHORT_MILLIS, TimeUnit.MILLISECONDS); + subscriber1.acknowledgeAsync(msg); + } + + // wait : so message can be eligible to to be evict from cache + Thread.sleep(maxMessageCacheRetentionTimeMillis); + + // 4. deactivate subscriber which has built the backlog + topicRef.checkBackloggedCursors(); + Thread.sleep(100); + + // 5. verify: active subscribers + Set activeSubscriber = new HashSet<>(); + ledger.getActiveCursors().forEach(c -> activeSubscriber.add(c.getName())); + assertTrue(activeSubscriber.contains(sub1)); + assertFalse(activeSubscriber.contains(sub2)); + + // 6. consume messages : at slower subscriber + for (int i = 0; i < totalMsgs; i++) { + msg = subscriber2.receive(RECEIVE_TIMEOUT_SHORT_MILLIS, TimeUnit.MILLISECONDS); + subscriber2.acknowledgeAsync(msg); + } + + topicRef.checkBackloggedCursors(); + + activeSubscriber.clear(); + ledger.getActiveCursors().forEach(c -> activeSubscriber.add(c.getName())); + + assertTrue(activeSubscriber.contains(sub1)); + assertTrue(activeSubscriber.contains(sub2)); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/MinimumBacklogCacheStrategyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/MinimumBacklogCacheStrategyTest.java new file mode 100644 index 0000000000000..3bb068def2806 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/MinimumBacklogCacheStrategyTest.java @@ -0,0 +1,158 @@ +/* + * 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.pulsar.broker.cache; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.lang.reflect.Field; +import java.util.concurrent.CountDownLatch; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.cache.PendingReadsManager; +import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.SubscriptionType; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker-api") +@Slf4j +public class MinimumBacklogCacheStrategyTest extends ProducerConsumerBase { + @BeforeMethod + @Override + protected void setup() throws Exception { + this.conf.setClusterName("test"); + internalSetup(); + producerBaseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + internalCleanup(); + } + + @Override + protected ServiceConfiguration getDefaultConf() { + ServiceConfiguration defaultConf = super.getDefaultConf(); + defaultConf.setManagedLedgerMinimumBacklogCursorsForCaching(2); + defaultConf.setManagedLedgerMinimumBacklogEntriesForCaching(10); + defaultConf.setManagedLedgerCacheEvictionTimeThresholdMillis(60 * 1000); + return defaultConf; + } + + /** + * Validates that backlog consumers cache the reads and reused by other backlog consumers while draining the + * backlog. + * + * @throws Exception + */ + @Test + public void testBacklogConsumerCacheReads() throws Exception { + log.info("-- Starting {} test --", methodName); + + final long totalMessages = 200; + final int receiverSize = 10; + final String topicName = "cache-read"; + final String sub1 = "sub"; + int totalSub = 10; + Consumer[] consumers = new Consumer[totalSub]; + + for (int i = 0; i < totalSub; i++) { + consumers[i] = pulsarClient.newConsumer().topic("persistent://my-property/my-ns/" + topicName) + .subscriptionName(sub1 + "-" + i).subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(receiverSize).subscribe(); + } + for (int i = 0; i < totalSub; i++) { + consumers[i].close(); + } + + final String topic = "persistent://my-property/my-ns/" + topicName; + ProducerBuilder producerBuilder = pulsarClient.newProducer().topic(topic); + + producerBuilder.enableBatching(false); + @Cleanup + Producer producer = producerBuilder.create(); + + PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) topicRef.getManagedLedger(); + Field cacheField = ManagedLedgerImpl.class.getDeclaredField("entryCache"); + cacheField.setAccessible(true); + RangeEntryCacheImpl entryCache = spy((RangeEntryCacheImpl) cacheField.get(ledger)); + cacheField.set(ledger, entryCache); + + Field pendingReadsManagerField = RangeEntryCacheImpl.class.getDeclaredField("pendingReadsManager"); + pendingReadsManagerField.setAccessible(true); + PendingReadsManager pendingReadsManager = (PendingReadsManager) pendingReadsManagerField.get(entryCache); + Field cacheFieldInManager = PendingReadsManager.class.getDeclaredField("rangeEntryCache"); + cacheFieldInManager.setAccessible(true); + cacheFieldInManager.set(pendingReadsManager, entryCache); + + // 2. Produce messages + for (int i = 0; i < totalMessages; i++) { + String message = "my-message-" + i; + producer.send(message.getBytes()); + } + ledger.checkCursorsToCacheEntries(); + + ledger.getCursors().forEach(cursor -> { + assertTrue(((ManagedCursorImpl) cursor).isCacheReadEntry()); + }); + + // 3. Consume messages + CountDownLatch latch = new CountDownLatch((int) (totalSub * totalMessages)); + for (int i = 0; i < totalSub; i++) { + consumers[i] = (Consumer) pulsarClient.newConsumer() + .topic("persistent://my-property/my-ns/" + topicName).subscriptionName(sub1 + "-" + i) + .subscriptionType(SubscriptionType.Shared).receiverQueueSize(receiverSize) + .messageListener((c, m) -> { + latch.countDown(); + try { + c.acknowledge(m); + } catch (PulsarClientException e) { + fail("failed to ack message"); + } + }).subscribe(); + } + + latch.await(); + + // Verify: EntryCache has been invalidated + verify(entryCache, atLeastOnce()).insert(any()); + + for (int i = 0; i < totalSub; i++) { + consumers[i].close(); + } + } + + +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index 5d6f0c519abc6..e05454a451ce3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -19,13 +19,7 @@ package org.apache.pulsar.client.api; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.verify; import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; import com.google.common.collect.Sets; import java.lang.reflect.Field; import java.util.HashMap; @@ -36,10 +30,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.cache.PendingReadsManager; -import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -1118,92 +1109,4 @@ public void testRelativeMessageRateLimitingThrottling(SubscriptionType subscript producer.close(); log.info("-- Exiting {} test --", methodName); } - - /** - * Validates that backlog consumers cache the reads and reused by other backlog consumers while draining the - * backlog. - * - * @throws Exception - */ - @Test - public void testBacklogConsumerCacheReads() throws Exception { - log.info("-- Starting {} test --", methodName); - - conf.setManagedLedgerMinimumBacklogCursorsForCaching(2); - conf.setManagedLedgerMinimumBacklogEntriesForCaching(10); - conf.setManagedLedgerCacheEvictionTimeThresholdMillis(60 * 1000); - restartBroker(); - final long totalMessages = 200; - final int receiverSize = 10; - final String topicName = "cache-read"; - final String sub1 = "sub"; - int totalSub = 10; - Consumer[] consumers = new Consumer[totalSub]; - - for (int i = 0; i < totalSub; i++) { - consumers[i] = pulsarClient.newConsumer().topic("persistent://my-property/my-ns/" + topicName) - .subscriptionName(sub1 + "-" + i).subscriptionType(SubscriptionType.Shared) - .receiverQueueSize(receiverSize).subscribe(); - } - for (int i = 0; i < totalSub; i++) { - consumers[i].close(); - } - - final String topic = "persistent://my-property/my-ns/" + topicName; - ProducerBuilder producerBuilder = pulsarClient.newProducer().topic(topic); - - producerBuilder.enableBatching(false); - @Cleanup - Producer producer = producerBuilder.create(); - - PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); - ManagedLedgerImpl ledger = (ManagedLedgerImpl) topicRef.getManagedLedger(); - Field cacheField = ManagedLedgerImpl.class.getDeclaredField("entryCache"); - cacheField.setAccessible(true); - RangeEntryCacheImpl entryCache = spy((RangeEntryCacheImpl) cacheField.get(ledger)); - cacheField.set(ledger, entryCache); - - Field pendingReadsManagerField = RangeEntryCacheImpl.class.getDeclaredField("pendingReadsManager"); - pendingReadsManagerField.setAccessible(true); - PendingReadsManager pendingReadsManager = (PendingReadsManager) pendingReadsManagerField.get(entryCache); - Field cacheFieldInManager = PendingReadsManager.class.getDeclaredField("rangeEntryCache"); - cacheFieldInManager.setAccessible(true); - cacheFieldInManager.set(pendingReadsManager, entryCache); - - // 2. Produce messages - for (int i = 0; i < totalMessages; i++) { - String message = "my-message-" + i; - producer.send(message.getBytes()); - } - ledger.checkCursorsToCacheEntries(); - - ledger.getCursors().forEach(cursor -> { - assertTrue(((ManagedCursorImpl) cursor).isCacheReadEntry()); - }); - - // 3. Consume messages - CountDownLatch latch = new CountDownLatch((int) (totalSub * totalMessages)); - for (int i = 0; i < totalSub; i++) { - consumers[i] = (Consumer) pulsarClient.newConsumer() - .topic("persistent://my-property/my-ns/" + topicName).subscriptionName(sub1 + "-" + i) - .subscriptionType(SubscriptionType.Shared).receiverQueueSize(receiverSize) - .messageListener((c, m) -> { - latch.countDown(); - try { - c.acknowledge(m); - } catch (PulsarClientException e) { - fail("failed to ack message"); - } - }).subscribe(); - } - - latch.await(); - - // Verify: EntryCache has been invalidated - verify(entryCache, atLeastOnce()).insert(any()); - - for (int i = 0; i < totalSub; i++) { - consumers[i].close(); - } - } }