diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt
index fd393cfec9b76..21422a41dcef9 100644
--- a/distribution/server/src/assemble/LICENSE.bin.txt
+++ b/distribution/server/src/assemble/LICENSE.bin.txt
@@ -338,12 +338,11 @@ The Apache Software License, Version 2.0
- io.prometheus-simpleclient_tracer_otel-0.16.0.jar
- io.prometheus-simpleclient_tracer_otel_agent-0.16.0.jar
* Prometheus exporter
- - io.prometheus-prometheus-metrics-config-1.3.1.jar
- - io.prometheus-prometheus-metrics-exporter-common-1.3.1.jar
- - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.1.jar
- - io.prometheus-prometheus-metrics-exposition-formats-1.3.1.jar
- - io.prometheus-prometheus-metrics-model-1.3.1.jar
- - io.prometheus-prometheus-metrics-shaded-protobuf-1.3.1.jar
+ - io.prometheus-prometheus-metrics-config-1.3.3.jar
+ - io.prometheus-prometheus-metrics-exporter-common-1.3.3.jar
+ - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.3.jar
+ - io.prometheus-prometheus-metrics-exposition-formats-1.3.3.jar
+ - io.prometheus-prometheus-metrics-model-1.3.3.jar
* Jakarta Bean Validation API
- jakarta.validation-jakarta.validation-api-2.0.2.jar
- javax.validation-validation-api-1.1.0.Final.jar
@@ -516,21 +515,21 @@ The Apache Software License, Version 2.0
* RoaringBitmap
- org.roaringbitmap-RoaringBitmap-1.2.0.jar
* OpenTelemetry
- - io.opentelemetry-opentelemetry-api-1.41.0.jar
- - io.opentelemetry-opentelemetry-api-incubator-1.41.0-alpha.jar
- - io.opentelemetry-opentelemetry-context-1.41.0.jar
- - io.opentelemetry-opentelemetry-exporter-common-1.41.0.jar
- - io.opentelemetry-opentelemetry-exporter-otlp-1.41.0.jar
- - io.opentelemetry-opentelemetry-exporter-otlp-common-1.41.0.jar
- - io.opentelemetry-opentelemetry-exporter-prometheus-1.41.0-alpha.jar
- - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.41.0.jar
- - io.opentelemetry-opentelemetry-sdk-1.41.0.jar
- - io.opentelemetry-opentelemetry-sdk-common-1.41.0.jar
- - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.41.0.jar
- - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.41.0.jar
- - io.opentelemetry-opentelemetry-sdk-logs-1.41.0.jar
- - io.opentelemetry-opentelemetry-sdk-metrics-1.41.0.jar
- - io.opentelemetry-opentelemetry-sdk-trace-1.41.0.jar
+ - io.opentelemetry-opentelemetry-api-1.44.1.jar
+ - io.opentelemetry-opentelemetry-api-incubator-1.44.1-alpha.jar
+ - io.opentelemetry-opentelemetry-context-1.44.1.jar
+ - io.opentelemetry-opentelemetry-exporter-common-1.44.1.jar
+ - io.opentelemetry-opentelemetry-exporter-otlp-1.44.1.jar
+ - io.opentelemetry-opentelemetry-exporter-otlp-common-1.44.1.jar
+ - io.opentelemetry-opentelemetry-exporter-prometheus-1.44.1-alpha.jar
+ - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.44.1.jar
+ - io.opentelemetry-opentelemetry-sdk-1.44.1.jar
+ - io.opentelemetry-opentelemetry-sdk-common-1.44.1.jar
+ - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.44.1.jar
+ - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.44.1.jar
+ - io.opentelemetry-opentelemetry-sdk-logs-1.44.1.jar
+ - io.opentelemetry-opentelemetry-sdk-metrics-1.44.1.jar
+ - io.opentelemetry-opentelemetry-sdk-trace-1.44.1.jar
- io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.33.6.jar
- io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.33.6-alpha.jar
- io.opentelemetry.instrumentation-opentelemetry-resources-1.33.6-alpha.jar
@@ -572,10 +571,8 @@ Protocol Buffers License
CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt
* Java Annotations API
- com.sun.activation-javax.activation-1.2.0.jar
- - javax.xml.bind-jaxb-api-2.3.1.jar
* Java Servlet API -- javax.servlet-javax.servlet-api-3.1.0.jar
* WebSocket Server API -- javax.websocket-javax.websocket-client-api-1.0.jar
- * Java Web Service REST API -- javax.ws.rs-javax.ws.rs-api-2.1.jar
* HK2 - Dependency Injection Kernel
- org.glassfish.hk2-hk2-api-2.6.1.jar
- org.glassfish.hk2-hk2-locator-2.6.1.jar
diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt
index 1601f32bb2b34..a21c272f91b1d 100644
--- a/distribution/shell/src/assemble/LICENSE.bin.txt
+++ b/distribution/shell/src/assemble/LICENSE.bin.txt
@@ -388,9 +388,9 @@ The Apache Software License, Version 2.0
- log4j-slf4j2-impl-2.23.1.jar
- log4j-web-2.23.1.jar
* OpenTelemetry
- - opentelemetry-api-1.41.0.jar
- - opentelemetry-api-incubator-1.41.0-alpha.jar
- - opentelemetry-context-1.41.0.jar
+ - opentelemetry-api-1.44.1.jar
+ - opentelemetry-api-incubator-1.44.1-alpha.jar
+ - opentelemetry-context-1.44.1.jar
* BookKeeper
- bookkeeper-common-allocator-4.17.1.jar
@@ -431,11 +431,8 @@ MIT License
CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt
* Java Annotations API
- - javax.annotation-api-1.3.2.jar
- javax.activation-1.2.0.jar
- - jaxb-api-2.3.1.jar
* WebSocket Server API -- javax.websocket-client-api-1.0.jar
- * Java Web Service REST API -- javax.ws.rs-api-2.1.jar
* HK2 - Dependency Injection Kernel
- hk2-api-2.6.1.jar
- hk2-locator-2.6.1.jar
diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml
index 0b79df8278f81..2a5536987cd42 100644
--- a/jetcd-core-shaded/pom.xml
+++ b/jetcd-core-shaded/pom.xml
@@ -45,6 +45,10 @@
io.netty
*
+
+ javax.annotation
+ javax.annotation-api
+
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/CachedEntry.java
similarity index 74%
rename from pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java
rename to managed-ledger/src/main/java/org/apache/bookkeeper/mledger/CachedEntry.java
index 4c416b6152cf8..eed88de2f0aac 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/CachedEntry.java
@@ -16,13 +16,11 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pulsar.client.util;
+package org.apache.bookkeeper.mledger;
-import java.util.UUID;
-import org.apache.commons.codec.digest.DigestUtils;
+import io.netty.util.ReferenceCounted;
-public class NameUtil {
- public static String generateRandomName() {
- return DigestUtils.sha1Hex(UUID.randomUUID().toString()).substring(0, 5);
- }
+public interface CachedEntry extends Entry, ReferenceCounted {
+ boolean matchesKey(Position key);
+ boolean canEvict();
}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/Entry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/Entry.java
index 428dcc9d97c76..733dea93bef44 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/Entry.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/Entry.java
@@ -32,8 +32,10 @@ public interface Entry {
/**
* @return the data
*/
+ @Deprecated
byte[] getData();
+ @Deprecated
byte[] getDataAndRelease();
/**
@@ -66,4 +68,8 @@ public interface Entry {
* of data reached to 0).
*/
boolean release();
+
+ default EntryReadCountHandler getReadCountHandler() {
+ return null;
+ }
}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/EntryReadCountHandler.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/EntryReadCountHandler.java
new file mode 100644
index 0000000000000..3bf919117d1f2
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/EntryReadCountHandler.java
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+public interface EntryReadCountHandler {
+ int getExpectedReadCount();
+ boolean incrementExpectedReadCount();
+ void markRead();
+}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java
index 042e03998696c..18396900ea6e6 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java
@@ -19,10 +19,13 @@
package org.apache.bookkeeper.mledger;
import com.google.common.collect.Range;
+import java.util.Collection;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
+import java.util.SortedSet;
import java.util.concurrent.CompletableFuture;
import java.util.function.Predicate;
import org.apache.bookkeeper.common.annotation.InterfaceAudience;
@@ -693,6 +696,7 @@ void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate
* @throws InterruptedException
* @throws ManagedLedgerException
*/
+ @Deprecated
List replayEntries(Set extends Position> positions)
throws InterruptedException, ManagedLedgerException;
@@ -708,8 +712,10 @@ List replayEntries(Set extends Position> positions)
* @return skipped positions
* set of positions which are already deleted/acknowledged and skipped while replaying them
*/
- Set extends Position> asyncReplayEntries(
- Set extends Position> positions, ReadEntriesCallback callback, Object ctx);
+ default Set extends Position> asyncReplayEntries(final Set extends Position> positions,
+ ReadEntriesCallback callback, Object ctx) {
+ return asyncReplayEntries(positions, callback, ctx, false);
+ }
/**
* Read the specified set of positions from ManagedLedger.
@@ -728,6 +734,28 @@ Set extends Position> asyncReplayEntries(
Set extends Position> asyncReplayEntries(
Set extends Position> positions, ReadEntriesCallback callback, Object ctx, boolean sortEntries);
+ /**
+ * Read the specified set of positions from ManagedLedger in ranges.
+ * This method is used to read entries in ranges to avoid reading all entries at once.
+ *
+ * @param positions
+ * set of positions to read
+ * @param callback
+ * callback object returning the result of each range
+ * @param ctx
+ * opaque context
+ * @param invokeCallbacksInOrder
+ * when true, the callback will be invoked in order of the positions, otherwise the callback will be
+ * invoked in the order of the completion of the range read.
+ * @return skipped positions
+ * set of positions which are already deleted/acknowledged and skipped while replaying them
+ */
+ default Set extends Position> asyncReplayEntriesInRanges(SortedSet extends Position> positions,
+ ManagedCursorReplayReadEntriesCallback callback,
+ Object ctx, boolean invokeCallbacksInOrder) {
+ throw new UnsupportedOperationException("Not implemented");
+ }
+
/**
* Close the cursor and releases the associated resources.
*
@@ -893,6 +921,24 @@ default ManagedCursorAttributes getManagedCursorAttributes() {
boolean isMessageDeleted(Position position);
+ /**
+ * Returns the deleted messages from the given positions.
+ * Implementation classes can override this method to provide a more efficient way to filter deleted messages.
+ *
+ * @param positions the positions to filter
+ * @return the set of deleted positions
+ */
+ default Set filterDeletedMessages(Collection extends Position> positions) {
+ Set deletedPositions = new HashSet<>();
+ // prefer for loop to avoid creating stream related instances
+ for (Position position : positions) {
+ if (isMessageDeleted(position)) {
+ deletedPositions.add(position);
+ }
+ }
+ return deletedPositions;
+ }
+
ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException;
long[] getBatchPositionAckSet(Position position);
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorReplayReadEntriesCallback.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorReplayReadEntriesCallback.java
new file mode 100644
index 0000000000000..5f7e9ad908fc3
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorReplayReadEntriesCallback.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.bookkeeper.mledger;
+
+import java.util.List;
+
+public interface ManagedCursorReplayReadEntriesCallback {
+ void readEntriesComplete(ManagedCursorReplayReadRange range, boolean isLast, List entries, Object ctx);
+
+ void readEntriesFailed(ManagedCursorReplayReadRange range, boolean isLast, ManagedLedgerException exception,
+ Object ctx);
+}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorReplayReadRange.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorReplayReadRange.java
new file mode 100644
index 0000000000000..9698e225e54b7
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorReplayReadRange.java
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+public interface ManagedCursorReplayReadRange extends Comparable {
+ int rangeIndex();
+
+ int totalRanges();
+
+ Position startPosition();
+
+ Position lastPosition();
+
+ default int size() {
+ if (startPosition().getLedgerId() != lastPosition().getLedgerId()) {
+ throw new IllegalStateException("Cannot calculate size for range spanning multiple ledgers");
+ }
+ return (int) (lastPosition().getEntryId() - startPosition().getEntryId() + 1);
+ }
+
+ @Override
+ default int compareTo(ManagedCursorReplayReadRange o) {
+ return Integer.compare(rangeIndex(), o.rangeIndex());
+ }
+}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java
index a1e1deb503e20..28508e4b617fd 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java
@@ -83,6 +83,9 @@ public class ManagedLedgerConfig {
@Getter
@Setter
private boolean cacheEvictionByMarkDeletedPosition = false;
+ @Getter
+ @Setter
+ private boolean cacheEvictionByExpectedReadCount = true;
private int minimumBacklogCursorsForCaching = 0;
private int minimumBacklogEntriesForCaching = 1000;
private int maxBacklogBetweenCursorsForCaching = 1000;
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java
index 386310b3ccbae..af538262ed44a 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java
@@ -61,6 +61,18 @@ public class ManagedLedgerFactoryConfig {
*/
private long managedLedgerMaxReadsInFlightSize = 0;
+ /**
+ * Maximum time to wait for acquiring permits for max reads in flight when managedLedgerMaxReadsInFlightSizeInMB is
+ * set (>0) and the limit is reached.
+ */
+ private long managedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis = 60000;
+
+ /**
+ * Maximum number of reads that can be queued for acquiring permits for max reads in flight when
+ * managedLedgerMaxReadsInFlightSizeInMB is set (>0) and the limit is reached.
+ */
+ private int managedLedgerMaxReadsInFlightPermitsAcquireQueueSize = 10000;
+
/**
* Whether trace managed ledger task execution time.
*/
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..8f52f26fe94c1
--- /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;
+
+abstract class AbstractEntryImpl> extends AbstractCASReferenceCounted
+ implements Entry, Comparable {
+ protected final Recycler.Handle recyclerHandle;
+ protected long timestamp;
+ protected long ledgerId;
+ protected long entryId;
+ ByteBuf data;
+ int length;
+ private Position position;
+ private Runnable onDeallocate;
+ protected EntryReadCountHandlerImpl readCountHandler;
+
+ 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;
+ // TODO: the buffer's memory size is the capacity, not the readableBytes
+ 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() {
+ 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 = 0;
+ timestamp = -1;
+ ledgerId = -1;
+ entryId = -1;
+ position = null;
+ readCountHandler = 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/CachedEntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/CachedEntryImpl.java
new file mode 100644
index 0000000000000..4454fd3f8ad35
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/CachedEntryImpl.java
@@ -0,0 +1,62 @@
+/*
+ * 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 org.apache.bookkeeper.mledger.CachedEntry;
+import org.apache.bookkeeper.mledger.Position;
+
+public 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, EntryReadCountHandlerImpl readCountHandler) {
+ CachedEntryImpl entry = RECYCLER.get();
+ entry.timestamp = System.nanoTime();
+ entry.ledgerId = position.getLedgerId();
+ entry.entryId = position.getEntryId();
+ entry.readCountHandler = readCountHandler;
+ entry.setDataBuffer(data.retainedDuplicate());
+ entry.setRefCnt(1);
+ return entry;
+ }
+
+ private CachedEntryImpl(Recycler.Handle recyclerHandle) {
+ super(recyclerHandle);
+ }
+
+ @Override
+ public boolean canEvict() {
+ if (readCountHandler != null) {
+ return readCountHandler.getExpectedReadCount() < 1;
+ }
+ return true;
+ }
+
+ @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/EntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java
index e0e2b859794b5..8cb16531ed3ac 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,11 @@
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.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 {
+public final class EntryImpl extends AbstractEntryImpl {
private static final Recycler RECYCLER = new Recycler() {
@Override
protected EntryImpl newObject(Handle handle) {
@@ -41,22 +34,15 @@ 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;
+ private boolean decreaseReadCountOnRelease;
- public static EntryImpl create(LedgerEntry ledgerEntry) {
+ public static EntryImpl create(LedgerEntry ledgerEntry, int expectedReadCount) {
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().retainedDuplicate());
+ entry.readCountHandler = EntryReadCountHandlerImpl.create(expectedReadCount);
entry.setRefCnt(1);
return entry;
}
@@ -67,18 +53,24 @@ 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;
}
public static EntryImpl create(long ledgerId, long entryId, ByteBuf data) {
+ return create(ledgerId, entryId, data, 0);
+ }
+
+ public static EntryImpl create(long ledgerId, long entryId, ByteBuf data, int expectedReadCount) {
EntryImpl entry = RECYCLER.get();
entry.timestamp = System.nanoTime();
entry.ledgerId = ledgerId;
entry.entryId = entryId;
- entry.data = data;
- entry.data.retain();
+ entry.setDataBuffer(data.retainedDuplicate());
+ if (expectedReadCount > 0) {
+ entry.readCountHandler = EntryReadCountHandlerImpl.create(expectedReadCount);
+ }
entry.setRefCnt(1);
return entry;
}
@@ -88,128 +80,42 @@ 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.retainedDuplicate());
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.readCountHandler = (EntryReadCountHandlerImpl) other.getReadCountHandler();
+ entry.setDataBuffer(other.getDataBuffer().retainedDuplicate());
entry.setRefCnt(1);
+ entry.decreaseReadCountOnRelease = true;
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;
+ super(recyclerHandle);
}
@Override
- public int compareTo(EntryImpl other) {
- if (this.ledgerId != other.ledgerId) {
- return this.ledgerId < other.ledgerId ? -1 : 1;
+ protected void beforeDeallocate() {
+ super.beforeDeallocate();
+ if (decreaseReadCountOnRelease && readCountHandler != null) {
+ readCountHandler.markRead();
}
-
- if (this.entryId != other.entryId) {
- return this.entryId < other.entryId ? -1 : 1;
- }
-
- return 0;
}
@Override
- public ReferenceCounted touch(Object hint) {
- return this;
+ protected void beforeRecycle() {
+ super.beforeRecycle();
+ decreaseReadCountOnRelease = false;
}
- @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;
+ public void setDecreaseReadCountOnRelease(boolean enabled) {
+ decreaseReadCountOnRelease = enabled;
}
}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryReadCountHandlerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryReadCountHandlerImpl.java
new file mode 100644
index 0000000000000..e959059b66268
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryReadCountHandlerImpl.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;
+
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import org.apache.bookkeeper.mledger.EntryReadCountHandler;
+
+public class EntryReadCountHandlerImpl implements EntryReadCountHandler {
+ private static AtomicIntegerFieldUpdater expectedReadCountUpdater =
+ AtomicIntegerFieldUpdater.newUpdater(EntryReadCountHandlerImpl.class, "expectedReadCount");
+
+ private volatile int expectedReadCount;
+
+ private EntryReadCountHandlerImpl(int expectedReadCount) {
+ this.expectedReadCount = expectedReadCount;
+ }
+
+ public int getExpectedReadCount() {
+ return expectedReadCount;
+ }
+
+ @Override
+ public boolean incrementExpectedReadCount() {
+ expectedReadCountUpdater.incrementAndGet(this);
+ return true;
+ }
+
+ @Override
+ public void markRead() {
+ expectedReadCountUpdater.decrementAndGet(this);
+ }
+
+ public static EntryReadCountHandlerImpl create(int expectedReadCount) {
+ return new EntryReadCountHandlerImpl(expectedReadCount);
+ }
+}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java
index ba901ece51c39..2f409e80eb296 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java
@@ -19,9 +19,10 @@
package org.apache.bookkeeper.mledger.impl;
import static java.util.Objects.requireNonNull;
-import java.util.ArrayList;
import java.util.Iterator;
import java.util.Map;
+import java.util.NavigableSet;
+import java.util.TreeSet;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.locks.StampedLock;
@@ -65,15 +66,31 @@ public static class CursorInfo {
long version;
}
- private static class Item {
+ private static class Item implements Comparable- {
final ManagedCursor cursor;
Position position;
- int idx;
- Item(ManagedCursor cursor, Position position, int idx) {
+ Item(ManagedCursor cursor, Position position) {
this.cursor = cursor;
this.position = position;
- this.idx = idx;
+ }
+
+ @Override
+ public int compareTo(ManagedCursorContainer.Item o) {
+ int retval = position.compareTo(o.position);
+ if (retval == 0) {
+ if (cursor == null) {
+ return 1;
+ }
+ if (o.cursor == null) {
+ return -1;
+ }
+ retval = cursor.getName().compareTo(o.cursor.getName());
+ }
+ if (retval == 0) {
+ retval = Integer.compare(System.identityHashCode(this), System.identityHashCode(o));
+ }
+ return retval;
}
}
@@ -138,7 +155,7 @@ public static long getNextVersion(long existingVersion) {
public ManagedCursorContainer() {}
// Used to keep track of slowest cursor.
- private final ArrayList
- heap = new ArrayList<>();
+ private final NavigableSet
- sortedByPosition = new TreeSet<>();
// Maps a cursor to its position in the heap
private final ConcurrentMap cursors = new ConcurrentSkipListMap<>();
@@ -160,13 +177,10 @@ public ManagedCursorContainer() {}
public void add(ManagedCursor cursor, Position position) {
long stamp = rwLock.writeLock();
try {
- Item item = new Item(cursor, position, position != null ? heap.size() : -1);
+ Item item = new Item(cursor, position);
cursors.put(cursor.getName(), item);
if (position != null) {
- heap.add(item);
- if (heap.size() > 1) {
- siftUp(item);
- }
+ sortedByPosition.add(item);
}
if (cursor.isDurable()) {
durableCursorCount++;
@@ -192,17 +206,8 @@ public boolean removeCursor(String name) {
try {
Item item = cursors.remove(name);
if (item != null) {
- if (item.idx >= 0) {
- if (heap.size() == 1) {
- heap.clear();
- } else {
- // Move the item to the right end of the heap to be removed
- Item lastItem = heap.get(heap.size() - 1);
- swap(item, lastItem);
- heap.remove(item.idx);
- // Update the heap
- siftDown(lastItem);
- }
+ if (item.position != null) {
+ sortedByPosition.remove(item);
}
if (item.cursor.isDurable()) {
durableCursorCount--;
@@ -235,26 +240,19 @@ public Pair cursorUpdated(ManagedCursor cursor, Position new
long stamp = rwLock.writeLock();
try {
Item item = cursors.get(cursor.getName());
- if (item == null || item.idx == -1) {
+ if (item == null) {
return null;
}
-
- Position previousSlowestConsumer = heap.get(0).position;
- item.position = newPosition;
- version = DataVersion.getNextVersion(version);
-
- if (heap.size() == 1) {
- return Pair.of(previousSlowestConsumer, item.position);
+ Position previousSlowestConsumer = !sortedByPosition.isEmpty() ? sortedByPosition.first().position : null;
+ if (item.position != null) {
+ sortedByPosition.remove(item);
}
-
- // When the cursor moves forward, we need to push it toward the
- // bottom of the tree and push it up if a reset was done
- if (item.idx == 0 || getParent(item).position.compareTo(item.position) <= 0) {
- siftDown(item);
- } else {
- siftUp(item);
+ item.position = newPosition;
+ if (newPosition != null) {
+ sortedByPosition.add(item);
}
- Position newSlowestConsumer = heap.get(0).position;
+ version = DataVersion.getNextVersion(version);
+ Position newSlowestConsumer = sortedByPosition.first().position;
return Pair.of(previousSlowestConsumer, newSlowestConsumer);
} finally {
rwLock.unlockWrite(stamp);
@@ -269,7 +267,7 @@ public Pair cursorUpdated(ManagedCursor cursor, Position new
public Position getSlowestReaderPosition() {
long stamp = rwLock.readLock();
try {
- return heap.isEmpty() ? null : heap.get(0).position;
+ return !sortedByPosition.isEmpty() ? sortedByPosition.first().position : null;
} finally {
rwLock.unlockRead(stamp);
}
@@ -278,7 +276,7 @@ public Position getSlowestReaderPosition() {
public ManagedCursor getSlowestReader() {
long stamp = rwLock.readLock();
try {
- return heap.isEmpty() ? null : heap.get(0).cursor;
+ return !sortedByPosition.isEmpty() ? sortedByPosition.first().cursor : null;
} finally {
rwLock.unlockRead(stamp);
}
@@ -291,10 +289,10 @@ public ManagedCursor getSlowestReader() {
public CursorInfo getCursorWithOldestPosition() {
long stamp = rwLock.readLock();
try {
- if (heap.isEmpty()) {
+ if (sortedByPosition.isEmpty()) {
return null;
} else {
- Item item = heap.get(0);
+ Item item = sortedByPosition.first();
return new CursorInfo(item.cursor, item.position, version);
}
} finally {
@@ -387,74 +385,29 @@ public void remove() {
};
}
- // //////////////////////
- /**
- * Push the item up towards the root of the tree (the lowest reading position).
- */
- private void siftUp(Item item) {
- Item parent = getParent(item);
- while (item.idx > 0 && parent.position.compareTo(item.position) > 0) {
- swap(item, parent);
- parent = getParent(item);
+ public int getNumberOfCursorsAtSamePositionOrBefore(ManagedCursor cursor) {
+ long stamp = rwLock.readLock();
+ try {
+ Item item = cursors.get(cursor.getName());
+ return item != null ? sortedByPosition.headSet(new Item(null, item.position)).size() : 0;
+ } finally {
+ rwLock.unlockRead(stamp);
}
}
- /**
- * Push the item down towards the bottom of the tree (the highest reading position).
- */
- private void siftDown(final Item item) {
- while (true) {
- Item j = null;
- Item right = getRight(item);
- if (right != null && right.position.compareTo(item.position) < 0) {
- Item left = getLeft(item);
- if (left != null && left.position.compareTo(right.position) < 0) {
- j = left;
- } else {
- j = right;
- }
- } else {
- Item left = getLeft(item);
- if (left != null && left.position.compareTo(item.position) < 0) {
- j = left;
- }
- }
-
- if (j != null) {
- swap(item, j);
- } else {
- break;
+ public int size() {
+ long stamp = rwLock.tryOptimisticRead();
+ int size = cursors.size();
+ if (!rwLock.validate(stamp)) {
+ // Fallback to read lock
+ stamp = rwLock.readLock();
+ try {
+ size = cursors.size();
+ } finally {
+ rwLock.unlockRead(stamp);
}
}
- }
-
- /**
- * Swap two items in the heap.
- */
- private void swap(Item item1, Item item2) {
- int idx1 = item1.idx;
- int idx2 = item2.idx;
-
- heap.set(idx2, item1);
- heap.set(idx1, item2);
-
- // Update the indexes too
- item1.idx = idx2;
- item2.idx = idx1;
- }
-
- private Item getParent(Item item) {
- return heap.get((item.idx - 1) / 2);
- }
-
- private Item getLeft(Item item) {
- int i = item.idx * 2 + 1;
- return i < heap.size() ? heap.get(i) : null;
- }
-
- private Item getRight(Item item) {
- int i = item.idx * 2 + 2;
- return i < heap.size() ? heap.get(i) : null;
+ return size;
}
}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java
index 478c6a1b37976..4a308c0568893 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java
@@ -35,6 +35,7 @@
import java.time.Clock;
import java.util.ArrayDeque;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
@@ -44,6 +45,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
+import java.util.SortedSet;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CountDownLatch;
@@ -80,6 +82,8 @@
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedCursorAttributes;
import org.apache.bookkeeper.mledger.ManagedCursorMXBean;
+import org.apache.bookkeeper.mledger.ManagedCursorReplayReadEntriesCallback;
+import org.apache.bookkeeper.mledger.ManagedCursorReplayReadRange;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
@@ -1550,12 +1554,6 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
* those messages can be removed by caller(Dispatcher)'s replay-list and it won't try to replay it again
*
*/
- @Override
- public Set extends Position> asyncReplayEntries(final Set extends Position> positions,
- ReadEntriesCallback callback, Object ctx) {
- return asyncReplayEntries(positions, callback, ctx, false);
- }
-
@Override
public Set extends Position> asyncReplayEntries(Set extends Position> positions,
ReadEntriesCallback callback, Object ctx, boolean sortEntries) {
@@ -1565,16 +1563,12 @@ public Set extends Position> asyncReplayEntries(Set extends Position> positi
return Collections.emptySet();
}
- // filters out messages which are already acknowledged
- Set alreadyAcknowledgedPositions = new HashSet<>();
- lock.readLock().lock();
- try {
- positions.stream().filter(this::internalIsMessageDeleted).forEach(alreadyAcknowledgedPositions::add);
- } finally {
- lock.readLock().unlock();
- }
+ Set alreadyAcknowledgedPositions = filterDeletedMessages(positions);
final int totalValidPositions = positions.size() - alreadyAcknowledgedPositions.size();
+ if (totalValidPositions == 0) {
+ return alreadyAcknowledgedPositions;
+ }
final AtomicReference exception = new AtomicReference<>();
ReadEntryCallback cb = new ReadEntryCallback() {
int pendingCallbacks = totalValidPositions;
@@ -1617,19 +1611,98 @@ public String toString() {
}
};
- positions.stream().filter(position -> !alreadyAcknowledgedPositions.contains(position))
- .forEach(p ->{
- if (p.compareTo(this.readPosition) == 0) {
- this.setReadPosition(this.readPosition.getNext());
- log.warn("[{}][{}] replayPosition{} equals readPosition{}," + " need set next readPosition",
- ledger.getName(), name, p, this.readPosition);
- }
- ledger.asyncReadEntry(p, cb, ctx);
- });
+ for (Position position : positions) {
+ if (!alreadyAcknowledgedPositions.contains(position)) {
+ if (position.compareTo(this.readPosition) == 0) {
+ this.setReadPosition(this.readPosition.getNext());
+ log.warn("[{}][{}] replayPosition{} equals readPosition{}," + " need set next readPosition",
+ ledger.getName(), name, position, this.readPosition);
+ }
+ ledger.asyncReadEntry(position, cb, ctx);
+ }
+ }
return alreadyAcknowledgedPositions;
}
+ @Override
+ public Set extends Position> asyncReplayEntriesInRanges(SortedSet extends Position> positions,
+ ManagedCursorReplayReadEntriesCallback callback,
+ Object callerCtx,
+ boolean invokeCallbacksInOrder) {
+ checkArgument(!positions.isEmpty(), "Positions to replay should not be empty");
+ Set alreadyAcknowledgedPositions = filterDeletedMessages(positions);
+ final int totalValidPositions = positions.size() - alreadyAcknowledgedPositions.size();
+ if (totalValidPositions == 0) {
+ return alreadyAcknowledgedPositions;
+ }
+ List ranges = toRanges(positions, alreadyAcknowledgedPositions);
+ if (isClosed()) {
+ callback.readEntriesFailed(ranges.get(0), true, new ManagedLedgerException
+ .CursorAlreadyClosedException("Cursor was already closed"), callerCtx);
+ } else {
+ AsyncCallbacks.ReadEntriesCallback cb =
+ new ManagedCursorReplayReadEntriesBatchCallbackImpl(ranges, invokeCallbacksInOrder, callback,
+ callerCtx);
+ for (ManagedCursorReplayReadRange range : ranges) {
+ ledger.asyncReadEntries(
+ OpReadEntry.create(this, range.startPosition(), range.size(), cb, range, null, null));
+ }
+ }
+ return alreadyAcknowledgedPositions;
+ }
+
+ private static List toRanges(SortedSet extends Position> positions,
+ Set alreadyAcknowledgedPositions) {
+ List> positionRanges = new ArrayList<>();
+ Position rangeStartPosition = null;
+ Position rangeLastPosition = null;
+ for (Position position : positions) {
+ if (!alreadyAcknowledgedPositions.contains(position)) {
+ if (rangeStartPosition == null) {
+ rangeStartPosition = position;
+ rangeLastPosition = position;
+ } else if (rangeLastPosition.getLedgerId() == position.getLedgerId()
+ && rangeLastPosition.getEntryId() + 1 == position.getEntryId()) {
+ rangeLastPosition = position;
+ } else {
+ positionRanges.add(Pair.of(rangeStartPosition, rangeLastPosition));
+ rangeStartPosition = position;
+ rangeLastPosition = position;
+ }
+ }
+ }
+ if (rangeStartPosition != null && rangeLastPosition != null) {
+ positionRanges.add(Pair.of(rangeStartPosition, rangeLastPosition));
+ }
+ List ranges = new ArrayList<>(positionRanges.size());
+ for (int i = 0; i < positionRanges.size(); i++) {
+ Pair range = positionRanges.get(i);
+ ranges.add(
+ new ManagedCursorReplayReadRangeImpl(i, positionRanges.size(), range.getLeft(), range.getRight()));
+ }
+ return ranges;
+ }
+
+ @Override
+ public Set filterDeletedMessages(Collection extends Position> positions) {
+ Set deletedMessages = new HashSet<>();
+ // acquire a read lock once for all the positions
+ lock.readLock().lock();
+ try {
+ // prefer for loop to avoid creating stream related instances
+ for (Position position : positions) {
+ // call the internal method to avoid acquiring read lock multiple times
+ if (internalIsMessageDeleted(position)) {
+ deletedMessages.add(position);
+ }
+ }
+ } finally {
+ lock.readLock().unlock();
+ }
+ return deletedMessages;
+ }
+
protected long getNumberOfEntries(Range range) {
long allEntries = ledger.getNumberOfEntries(range);
@@ -3721,14 +3794,14 @@ public int applyMaxSizeCap(int maxEntries, long maxSizeBytes) {
double avgEntrySize = ledger.getStats().getEntrySizeAverage();
if (!Double.isFinite(avgEntrySize)) {
- // We don't have yet any stats on the topic entries. Let's try to use the cursor avg size stats
- avgEntrySize = (double) entriesReadSize / (double) entriesReadCount;
- }
-
- if (!Double.isFinite(avgEntrySize)) {
- // If we still don't have any information, it means this is the first time we attempt reading
- // and there are no writes. Let's start with 1 to avoid any overflow and start the avg stats
- return 1;
+ if (entriesReadCount != 0) {
+ // We don't have yet any stats on the topic entries. Let's try to use the cursor avg size stats
+ avgEntrySize = (double) entriesReadSize / (double) entriesReadCount;
+ } else {
+ // If we still don't have any information, it means this is the first time we attempt reading
+ // and there are no writes. Let's start with 1 to avoid any overflow and start the avg stats
+ return 1;
+ }
}
int maxEntriesBasedOnSize = (int) (maxSizeBytes / avgEntrySize);
@@ -3831,4 +3904,14 @@ public ManagedLedgerInternalStats.CursorStats getCursorStats() {
cs.properties = getProperties();
return cs;
}
+
+ public int getNumberOfCursorsAtSamePositionOrBefore() {
+ if (ledger.getConfig().isCacheEvictionByExpectedReadCount()) {
+ return ledger.getNumberOfCursorsAtSamePositionOrBefore(this);
+ } else if (isCacheReadEntry()) {
+ return 1;
+ } else {
+ return 0;
+ }
+ }
}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorReplayReadEntriesBatchCallbackImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorReplayReadEntriesBatchCallbackImpl.java
new file mode 100644
index 0000000000000..e52384f5f2dbb
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorReplayReadEntriesBatchCallbackImpl.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.bookkeeper.mledger.impl;
+
+import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursorReplayReadEntriesCallback;
+import org.apache.bookkeeper.mledger.ManagedCursorReplayReadRange;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+
+@ToString
+@Slf4j
+class ManagedCursorReplayReadEntriesBatchCallbackImpl implements AsyncCallbacks.ReadEntriesCallback {
+ private final List ranges;
+ private final boolean invokeCallbacksInOrder;
+ private final ManagedCursorReplayReadEntriesCallback callback;
+ private final Object callerCtx;
+ int pendingCallbacks;
+ SortedMap> pendingResults;
+ int nextResultIndex;
+ ManagedLedgerException exception;
+ ManagedCursorReplayReadRange failedRange;
+
+ public ManagedCursorReplayReadEntriesBatchCallbackImpl(List ranges,
+ boolean invokeCallbacksInOrder,
+ ManagedCursorReplayReadEntriesCallback callback,
+ Object callerCtx) {
+ this.ranges = ranges;
+ this.invokeCallbacksInOrder = invokeCallbacksInOrder;
+ this.callback = callback;
+ this.callerCtx = callerCtx;
+ pendingCallbacks = ranges.size();
+ pendingResults = new TreeMap<>();
+ nextResultIndex = 0;
+ }
+
+ @Override
+ public synchronized void readEntriesComplete(List entries, Object ctx) {
+ if (invokeCallbacksInOrder && exception != null) {
+ // if there is already a previous failure, we should release the entry straight away
+ // and not add it to the list
+ entries.forEach(Entry::release);
+ if (--pendingCallbacks == 0) {
+ callback.readEntriesFailed(failedRange, true, exception, callerCtx);
+ }
+ } else {
+ ManagedCursorReplayReadRange range = (ManagedCursorReplayReadRange) ctx;
+ if (!invokeCallbacksInOrder || nextResultIndex == range.rangeIndex()) {
+ if (!invokeCallbacksInOrder) {
+ nextResultIndex++;
+ }
+ boolean isLast = (--pendingCallbacks == 0);
+ callback.readEntriesComplete(range, isLast, entries, callerCtx);
+ if (invokeCallbacksInOrder) {
+ finishPossiblePendingResults();
+ }
+ } else {
+ pendingResults.put(range, entries);
+ }
+ }
+ }
+
+ private void finishPossiblePendingResults() {
+ while (!pendingResults.isEmpty()) {
+ ManagedCursorReplayReadRange range = pendingResults.firstKey();
+ if (range.rangeIndex() == nextResultIndex) {
+ nextResultIndex++;
+ List entries = pendingResults.remove(range);
+ boolean isLast = (--pendingCallbacks == 0);
+ callback.readEntriesComplete(range, isLast, entries, callerCtx);
+ } else {
+ break;
+ }
+ }
+ }
+
+ @Override
+ public synchronized void readEntriesFailed(ManagedLedgerException mle, Object ctx) {
+ failedRange = (ManagedCursorReplayReadRange) ctx;
+ exception = mle;
+ if (!(mle instanceof ManagedLedgerException.CursorAlreadyClosedException)) {
+ log.warn("Error while replaying entries for range {}", mle, failedRange);
+ }
+ // when invoking in order is required, fail any pending entries
+ if (!pendingResults.isEmpty()) {
+ for (List entries : pendingResults.values()) {
+ entries.forEach(Entry::release);
+ pendingCallbacks--;
+ }
+ pendingResults.clear();
+ }
+ if (--pendingCallbacks == 0) {
+ callback.readEntriesFailed(failedRange, true, exception, callerCtx);
+ }
+ }
+}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorReplayReadRangeImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorReplayReadRangeImpl.java
new file mode 100644
index 0000000000000..f97e4c239779f
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorReplayReadRangeImpl.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.bookkeeper.mledger.impl;
+
+import org.apache.bookkeeper.mledger.ManagedCursorReplayReadRange;
+import org.apache.bookkeeper.mledger.Position;
+
+public record ManagedCursorReplayReadRangeImpl(int rangeIndex, int totalRanges, Position startPosition,
+ Position lastPosition) implements ManagedCursorReplayReadRange {
+}
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 f546a487f84be..496cb05bdb66f 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 =
@@ -228,7 +242,7 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore,
compressionConfigForManagedCursorInfo);
this.config = config;
this.mbean = new ManagedLedgerFactoryMBeanImpl(this);
- this.entryCacheManager = new RangeEntryCacheManagerImpl(this, openTelemetry);
+ this.entryCacheManager = new RangeEntryCacheManagerImpl(this, scheduledExecutor, openTelemetry);
this.statsTask = scheduledExecutor.scheduleWithFixedDelay(catchingAndLoggingThrowables(this::refreshStats),
0, config.getStatsPeriodSeconds(), TimeUnit.SECONDS);
this.flushCursorsTask = scheduledExecutor.scheduleAtFixedRate(catchingAndLoggingThrowables(this::flushCursors),
@@ -298,17 +312,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);
+ }
}
/**
@@ -324,7 +344,7 @@ public Map getManagedLedgers() {
@Override
public ManagedLedger open(String name) throws InterruptedException, ManagedLedgerException {
- return open(name, new ManagedLedgerConfig());
+ return open(name, defaultManagedLedgerConfig);
}
@Override
@@ -360,7 +380,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 926e7c7f8108e..f837338a49c14 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
@@ -65,6 +65,7 @@
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.IntSupplier;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@@ -162,6 +163,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;
@@ -494,7 +496,7 @@ protected ManagedLedgerInterceptor.LastEntryHandle createLastEntryHandle(LedgerH
entries.getEntry(lh.getLastAddConfirmed());
if (ledgerEntry != null) {
promise.complete(
- Optional.of(EntryImpl.create(ledgerEntry)));
+ Optional.of(EntryImpl.create(ledgerEntry, 0)));
} else {
promise.complete(Optional.empty());
}
@@ -2162,6 +2164,7 @@ protected void asyncReadEntry(ReadHandle ledger, Position position, ReadEntryCal
protected void asyncReadEntry(ReadHandle ledger, long firstEntry, long lastEntry, OpReadEntry opReadEntry,
Object ctx) {
+ IntSupplier expectedReadCount = () -> opReadEntry.cursor.getNumberOfCursorsAtSamePositionOrBefore();
if (config.getReadEntryTimeoutSeconds() > 0) {
// set readOpCount to uniquely validate if ReadEntryCallbackWrapper is already recycled
long readOpCount = READ_OP_COUNT_UPDATER.incrementAndGet(this);
@@ -2169,11 +2172,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, expectedReadCount, readCallback, readOpCount);
} else {
- entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, opReadEntry.cursor.isCacheReadEntry(), opReadEntry,
- ctx);
+ entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount, opReadEntry, ctx);
}
}
@@ -2332,25 +2333,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) {
+ if (entryCache.getSize() <= 0 || config.isCacheEvictionByExpectedReadCount()) {
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);
+ }
+ });
}
}
@@ -2385,7 +2396,7 @@ void onCursorMarkDeletePositionUpdated(ManagedCursorImpl cursor, Position newPos
private void updateActiveCursor(ManagedCursorImpl cursor, Position newPosition) {
Pair slowestPositions = activeCursors.cursorUpdated(cursor, newPosition);
- if (slowestPositions != null
+ if (!config.isCacheEvictionByExpectedReadCount() && slowestPositions != null
&& !slowestPositions.getLeft().equals(slowestPositions.getRight())) {
invalidateEntriesUpToSlowestReaderPosition();
}
@@ -2868,7 +2879,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());
+ });
}
}
@@ -3831,6 +3844,10 @@ private void deactivateCursorByName(String cursorName) {
}
}
+ public int getNumberOfCursorsAtSamePositionOrBefore(ManagedCursor cursor) {
+ return activeCursors.getNumberOfCursorsAtSamePositionOrBefore(cursor);
+ }
+
public void removeWaitingCursor(ManagedCursor cursor) {
this.waitingCursors.remove(cursor);
@@ -4565,7 +4582,7 @@ public boolean checkInactiveLedgerAndRollOver() {
public void checkCursorsToCacheEntries() {
- if (minBacklogCursorsForCaching < 1) {
+ if (minBacklogCursorsForCaching < 1 || config.isCacheEvictionByExpectedReadCount()) {
return;
}
Iterator it = cursors.iterator();
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java
index 036ce9223e89d..10a5e204167ae 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java
@@ -240,13 +240,16 @@ public void run() {
long ledgerId = ledger != null ? ledger.getId() : ((Position) ctx).getLedgerId();
// Don't insert to the entry cache for the ShadowManagedLedger
- if (!(ml instanceof ShadowManagedLedgerImpl) && ml.hasActiveCursors()) {
- // Avoid caching entries if no cursor has been created
- EntryImpl entry = EntryImpl.create(ledgerId, entryId, data);
- // EntryCache.insert: duplicates entry by allocating new entry and data. so, recycle entry after calling
- // insert
- ml.entryCache.insert(entry);
- entry.release();
+ if (!(ml instanceof ShadowManagedLedgerImpl)) {
+ int expectedReadCount = ml.getActiveCursors().size();
+ if (expectedReadCount > 0) {
+ // Avoid caching entries if no cursor has been created
+ EntryImpl entry = EntryImpl.create(ledgerId, entryId, data, expectedReadCount);
+ // EntryCache.insert: duplicates entry by allocating new entry and data. so, recycle entry after calling
+ // insert
+ ml.entryCache.insert(entry);
+ entry.release();
+ }
}
Position lastEntry = PositionFactory.create(ledgerId, entryId);
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..db2194db252d5 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
@@ -18,18 +18,19 @@
*/
package org.apache.bookkeeper.mledger.impl.cache;
+import java.util.function.IntSupplier;
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.CachedEntry;
+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
@@ -46,7 +47,7 @@ public interface EntryCache extends Comparable {
* the entry to be cached
* @return whether the entry was inserted in cache
*/
- boolean insert(EntryImpl entry);
+ CachedEntry insert(Entry entry);
/**
* Remove from cache all the entries related to a ledger up to lastPosition excluded.
@@ -56,8 +57,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 +70,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.
*
@@ -91,14 +81,15 @@ public interface EntryCache extends Comparable {
* the first entry to read (inclusive)
* @param lastEntry
* the last entry to read (inclusive)
- * @param shouldCacheEntry
- * whether the read entry should be cached
+ * @param expectedReadCount
+ * resolves the expected read count for the given entry. When the expected read count is >0, the entry
+ * can be cached.
* @param callback
* the callback object that will be notified when read is done
* @param ctx
* the context object
*/
- void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry,
+ void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount,
ReadEntriesCallback callback, Object ctx);
/**
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..ffaab09849656 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
@@ -22,16 +22,17 @@
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
+import java.util.function.IntSupplier;
import org.apache.bookkeeper.client.api.LedgerEntry;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.CachedEntry;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.Position;
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,8 +52,8 @@ public String getName() {
}
@Override
- public boolean insert(EntryImpl entry) {
- return false;
+ public CachedEntry insert(Entry entry) {
+ return null;
}
@Override
@@ -68,16 +69,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, IntSupplier expectedReadCount,
final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) {
ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry).thenAcceptAsync(
ledgerEntries -> {
@@ -86,7 +78,8 @@ 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 =
+ RangeEntryCacheManagerImpl.create(e, interceptor, 0);
entries.add(entry);
totalSize += entry.getLength();
}
@@ -119,7 +112,8 @@ 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 =
+ RangeEntryCacheManagerImpl.create(ledgerEntry, interceptor, 0);
ml.getMbean().recordReadEntriesOpsCacheMisses(1, returnEntry.getLength());
ml.getFactory().getMbean().recordCacheMiss(1, returnEntry.getLength());
@@ -139,10 +133,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..188f6590e6d83 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,8 @@ public interface EntryCacheManager {
void updateCacheEvictionWatermark(double cacheEvictionWatermark);
double getCacheEvictionWatermark();
+
+ EntryCachesEvictionHandler getEvictionHandler();
+
+ void doCacheEviction(long maxTimestamp);
}
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/EntryCachesEvictionHandler.java
similarity index 68%
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/EntryCachesEvictionHandler.java
index 41284825fa76f..3883167ebbd3e 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/EntryCachesEvictionHandler.java
@@ -18,20 +18,17 @@
*/
package org.apache.bookkeeper.mledger.impl.cache;
-import java.util.List;
+import org.apache.commons.lang3.tuple.Pair;
+
+public interface EntryCachesEvictionHandler {
+ void invalidateEntriesBeforeTimestampNanos(long timestamp);
-/**
- * Cache eviction policy abstraction interface.
- *
- */
-public interface EntryCacheEvictionPolicy {
/**
- * Perform the cache eviction of at least sizeToFree bytes on the supplied list of caches.
+ * Force the cache to drop entries to free space.
*
- * @param caches
- * the list of caches to consider
* @param sizeToFree
- * the minimum size in bytes to be freed
+ * the total memory size to free
+ * @return a pair containing the number of entries evicted and their total size
*/
- void doEviction(List caches, long sizeToFree);
+ Pair evictEntries(long sizeToFree);
}
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java
index c87807b86631b..3a3b26ad994e0 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java
@@ -22,12 +22,18 @@
import io.opentelemetry.api.OpenTelemetry;
import io.opentelemetry.api.metrics.ObservableLongCounter;
import io.prometheus.client.Gauge;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
import lombok.AllArgsConstructor;
import lombok.ToString;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.opentelemetry.Constants;
import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization;
import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric;
+import org.jctools.queues.SpscArrayQueue;
@Slf4j
public class InflightReadsLimiter implements AutoCloseable {
@@ -58,16 +64,41 @@ public class InflightReadsLimiter implements AutoCloseable {
private final long maxReadsInFlightSize;
private long remainingBytes;
+ private final long acquireTimeoutMillis;
+ private final ScheduledExecutorService timeOutExecutor;
+ private final boolean enabled;
- public InflightReadsLimiter(long maxReadsInFlightSize, OpenTelemetry openTelemetry) {
- if (maxReadsInFlightSize <= 0) {
+ @AllArgsConstructor
+ @ToString
+ static class Handle {
+ final long permits;
+ final long creationTime;
+ final boolean success;
+ }
+
+ record QueuedHandle(Handle handle, Consumer callback) {
+ }
+
+ private final Queue queuedHandles;
+ private boolean timeoutCheckRunning = false;
+
+ public InflightReadsLimiter(long maxReadsInFlightSize, int maxReadsInFlightAcquireQueueSize,
+ long acquireTimeoutMillis, ScheduledExecutorService timeOutExecutor,
+ OpenTelemetry openTelemetry) {
+ this.maxReadsInFlightSize = maxReadsInFlightSize;
+ this.remainingBytes = maxReadsInFlightSize;
+ this.acquireTimeoutMillis = acquireTimeoutMillis;
+ this.timeOutExecutor = timeOutExecutor;
+ if (maxReadsInFlightSize > 0) {
+ enabled = true;
+ this.queuedHandles = new SpscArrayQueue<>(maxReadsInFlightAcquireQueueSize);
+ } else {
+ enabled = false;
+ this.queuedHandles = null;
// set it to -1 in order to show in the metrics that the metric is not available
PULSAR_ML_READS_BUFFER_SIZE.set(-1);
PULSAR_ML_READS_AVAILABLE_BUFFER_SIZE.set(-1);
}
- this.maxReadsInFlightSize = maxReadsInFlightSize;
- this.remainingBytes = maxReadsInFlightSize;
-
var meter = openTelemetry.getMeter(Constants.BROKER_INSTRUMENTATION_SCOPE_NAME);
inflightReadsLimitCounter = meter.counterBuilder(INFLIGHT_READS_LIMITER_LIMIT_METRIC_NAME)
.setDescription("Maximum number of bytes that can be retained by managed ledger data read from storage "
@@ -102,71 +133,145 @@ public void close() {
inflightReadsUsageCounter.close();
}
- @AllArgsConstructor
- @ToString
- static class Handle {
- final long acquiredPermits;
- final boolean success;
- final int trials;
+ private static final Handle DISABLED = new Handle(0, 0, true);
+ private static final Optional DISABLED_OPTIONAL = Optional.of(DISABLED);
- final long creationTime;
+ /**
+ * Acquires permits from the limiter. If the limiter is disabled, it will immediately return a successful handle.
+ * If permits are available, it will return a handle with the acquired permits. If no permits are available,
+ * it will return an empty optional and the callback will be called when permits become available or when the
+ * acquire timeout is reached. The success field in the handle passed to the callback will be false if the acquire
+ * operation times out. The callback should be non-blocking and run on a desired executor handled within the
+ * callback itself.
+ *
+ * A successful handle will have the success field set to true, and the caller must call release with the handle
+ * when the permits are no longer needed.
+ *
+ * If an unsuccessful handle is returned immediately, it means that the queue limit has been reached and the
+ * callback will not be called. The caller should fail the read operation in this case to apply backpressure.
+ *
+ * @param permits the number of permits to acquire
+ * @param callback the callback to be called when the permits are acquired or timed out
+ * @return an optional handle that contains the permits if acquired, otherwise an empty optional
+ */
+ public Optional acquire(long permits, Consumer callback) {
+ if (isDisabled()) {
+ return DISABLED_OPTIONAL;
+ }
+ return internalAcquire(permits, callback);
}
- private static final Handle DISABLED = new Handle(0, true, 0, -1);
+ private synchronized Optional internalAcquire(long permits, Consumer callback) {
+ Handle handle = new Handle(permits, System.currentTimeMillis(), true);
+ if (remainingBytes >= permits) {
+ remainingBytes -= permits;
+ if (log.isDebugEnabled()) {
+ log.debug("acquired permits: {}, creationTime: {}, remainingBytes:{}", permits, handle.creationTime,
+ remainingBytes);
+ }
+ updateMetrics();
+ return Optional.of(handle);
+ } else {
+ if (queuedHandles.offer(new QueuedHandle(handle, callback))) {
+ scheduleTimeOutCheck(acquireTimeoutMillis);
+ return Optional.empty();
+ } else {
+ log.warn("Failed to queue handle for acquiring permits: {}, creationTime: {}, remainingBytes:{}",
+ permits, handle.creationTime, remainingBytes);
+ return Optional.of(new Handle(0, handle.creationTime, false));
+ }
+ }
+ }
- Handle acquire(long permits, Handle current) {
- if (maxReadsInFlightSize <= 0) {
- // feature is disabled
- return DISABLED;
+ private synchronized void scheduleTimeOutCheck(long delayMillis) {
+ if (acquireTimeoutMillis <= 0) {
+ return;
}
- synchronized (this) {
- try {
- if (current == null) {
- if (remainingBytes == 0) {
- return new Handle(0, false, 1, System.currentTimeMillis());
- }
- if (remainingBytes >= permits) {
- remainingBytes -= permits;
- return new Handle(permits, true, 1, System.currentTimeMillis());
- } else {
- long possible = remainingBytes;
- remainingBytes = 0;
- return new Handle(possible, false, 1, System.currentTimeMillis());
- }
+ if (!timeoutCheckRunning) {
+ timeoutCheckRunning = true;
+ timeOutExecutor.schedule(this::timeoutCheck, delayMillis, TimeUnit.MILLISECONDS);
+ }
+ }
+
+ private synchronized void timeoutCheck() {
+ timeoutCheckRunning = false;
+ long delay = 0;
+ while (true) {
+ QueuedHandle queuedHandle = queuedHandles.peek();
+ if (queuedHandle != null) {
+ long age = System.currentTimeMillis() - queuedHandle.handle.creationTime;
+ if (age >= acquireTimeoutMillis) {
+ // remove the peeked handle from the queue
+ queuedHandles.poll();
+ handleTimeout(queuedHandle);
} else {
- if (current.trials >= 4 && current.acquiredPermits > 0) {
- remainingBytes += current.acquiredPermits;
- return new Handle(0, false, 1, current.creationTime);
- }
- if (remainingBytes == 0) {
- return new Handle(current.acquiredPermits, false, current.trials + 1,
- current.creationTime);
- }
- long needed = permits - current.acquiredPermits;
- if (remainingBytes >= needed) {
- remainingBytes -= needed;
- return new Handle(permits, true, current.trials + 1, current.creationTime);
- } else {
- long possible = remainingBytes;
- remainingBytes = 0;
- return new Handle(current.acquiredPermits + possible, false,
- current.trials + 1, current.creationTime);
- }
+ delay = acquireTimeoutMillis - age;
+ break;
}
- } finally {
- updateMetrics();
+ } else {
+ break;
}
}
+ if (delay > 0) {
+ scheduleTimeOutCheck(delay);
+ }
+ }
+
+ private void handleTimeout(QueuedHandle queuedHandle) {
+ if (log.isDebugEnabled()) {
+ log.debug("timed out queued permits: {}, creationTime: {}, remainingBytes:{}",
+ queuedHandle.handle.permits, queuedHandle.handle.creationTime, remainingBytes);
+ }
+ queuedHandle.callback.accept(new Handle(0, queuedHandle.handle.creationTime, false));
}
- void release(Handle handle) {
+ /**
+ * Releases permits back to the limiter. If the handle is disabled, this method will be a no-op.
+ *
+ * @param handle the handle containing the permits to release
+ */
+ public void release(Handle handle) {
if (handle == DISABLED) {
return;
}
- synchronized (this) {
- remainingBytes += handle.acquiredPermits;
- updateMetrics();
+ internalRelease(handle);
+ }
+
+ private synchronized void internalRelease(Handle handle) {
+ if (log.isDebugEnabled()) {
+ log.debug("release permits: {}, creationTime: {}, remainingBytes:{}", handle.permits,
+ handle.creationTime, getRemainingBytes());
+ }
+ remainingBytes += handle.permits;
+ while (true) {
+ QueuedHandle queuedHandle = queuedHandles.peek();
+ if (queuedHandle != null) {
+ if (remainingBytes >= queuedHandle.handle.permits) {
+ // remove the peeked handle from the queue
+ queuedHandles.poll();
+ handleQueuedHandle(queuedHandle);
+ } else if (acquireTimeoutMillis > 0
+ && System.currentTimeMillis() - queuedHandle.handle.creationTime > acquireTimeoutMillis) {
+ // remove the peeked handle from the queue
+ queuedHandles.poll();
+ handleTimeout(queuedHandle);
+ } else {
+ break;
+ }
+ } else {
+ break;
+ }
+ }
+ updateMetrics();
+ }
+
+ private void handleQueuedHandle(QueuedHandle queuedHandle) {
+ remainingBytes -= queuedHandle.handle.permits;
+ if (log.isDebugEnabled()) {
+ log.debug("acquired queued permits: {}, creationTime: {}, remainingBytes:{}",
+ queuedHandle.handle.permits, queuedHandle.handle.creationTime, remainingBytes);
}
+ queuedHandle.callback.accept(queuedHandle.handle);
}
private synchronized void updateMetrics() {
@@ -175,8 +280,6 @@ private synchronized void updateMetrics() {
}
public boolean isDisabled() {
- return maxReadsInFlightSize <= 0;
+ return !enabled;
}
-
-
-}
+}
\ No newline at end of file
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 8b2f3e25f1cbb..7462a7ee6889b 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
@@ -26,6 +26,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.IntSupplier;
import lombok.AllArgsConstructor;
import lombok.Value;
import lombok.extern.slf4j.Slf4j;
@@ -232,9 +233,9 @@ public PendingRead(PendingReadKey key,
this.ledgerCache = ledgerCache;
}
- private List keepEntries(List list, long startEntry, long endEntry) {
- List result = new ArrayList<>((int) (endEntry - startEntry));
- for (EntryImpl entry : list) {
+ private List keepEntries(List list, long startEntry, long endEntry) {
+ List result = new ArrayList<>((int) (endEntry - startEntry));
+ for (Entry entry : list) {
long entryId = entry.getEntryId();
if (startEntry <= entryId && entryId <= endEntry) {
result.add(entry);
@@ -245,7 +246,7 @@ private List keepEntries(List list, long startEntry, long
return result;
}
- public void attach(CompletableFuture
> handle) {
+ public void attach(CompletableFuture> handle) {
// when the future is done remove this from the map
// new reads will go to a new instance
// this is required because we are going to do refcount management
@@ -263,31 +264,31 @@ public void attach(CompletableFuture> handle) {
synchronized (PendingRead.this) {
if (callbacks.size() == 1) {
ReadEntriesCallbackWithContext first = callbacks.get(0);
- if (first.startEntry == key.startEntry
- && first.endEntry == key.endEntry) {
+ List entries;
+ if (first.startEntry == key.startEntry && first.endEntry == key.endEntry) {
// perfect match, no copy, this is the most common case
- first.callback.readEntriesComplete((List) entriesToReturn,
- first.ctx);
+ entries = entriesToReturn;
} else {
- first.callback.readEntriesComplete(
- (List) keepEntries(entriesToReturn, first.startEntry, first.endEntry),
- first.ctx);
+ entries = keepEntries(entriesToReturn, first.startEntry, first.endEntry);
}
+ first.callback.readEntriesComplete(entries, first.ctx);
} else {
for (ReadEntriesCallbackWithContext callback : callbacks) {
long callbackStartEntry = callback.startEntry;
long callbackEndEntry = callback.endEntry;
- List copy = new ArrayList<>((int) (callbackEndEntry - callbackStartEntry + 1));
- for (EntryImpl entry : entriesToReturn) {
+ List copy = new ArrayList<>((int) (callbackEndEntry - callbackStartEntry + 1));
+ for (Entry entry : entriesToReturn) {
long entryId = entry.getEntryId();
if (callbackStartEntry <= entryId && entryId <= callbackEndEntry) {
EntryImpl entryCopy = EntryImpl.create(entry);
copy.add(entryCopy);
}
}
- callback.callback.readEntriesComplete((List) copy, callback.ctx);
+ callback.callback.readEntriesComplete(copy, callback.ctx);
}
- for (EntryImpl entry : entriesToReturn) {
+ for (Entry entry : entriesToReturn) {
+ // don't decrease the read count when these entries are released
+ ((EntryImpl) entry).setDecreaseReadCountOnRelease(false);
entry.release();
}
}
@@ -314,7 +315,7 @@ synchronized boolean addListener(AsyncCallbacks.ReadEntriesCallback callback,
}
- void readEntries(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry,
+ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount,
final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) {
final PendingReadKey key = new PendingReadKey(firstEntry, lastEntry);
@@ -362,7 +363,7 @@ public void readEntriesFailed(ManagedLedgerException exception,
};
rangeEntryCache.asyncReadEntry0(lh,
missingOnRight.startEntry, missingOnRight.endEntry,
- shouldCacheEntry, readFromRightCallback, null);
+ expectedReadCount, readFromRightCallback, null, false);
}
@Override
@@ -372,7 +373,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object dummyCtx4
}
};
rangeEntryCache.asyncReadEntry0(lh, missingOnLeft.startEntry, missingOnLeft.endEntry,
- shouldCacheEntry, readFromLeftCallback, null);
+ expectedReadCount, readFromLeftCallback, null, false);
} else if (missingOnLeft != null) {
AsyncCallbacks.ReadEntriesCallback readFromLeftCallback =
new AsyncCallbacks.ReadEntriesCallback() {
@@ -395,7 +396,7 @@ public void readEntriesFailed(ManagedLedgerException exception,
}
};
rangeEntryCache.asyncReadEntry0(lh, missingOnLeft.startEntry, missingOnLeft.endEntry,
- shouldCacheEntry, readFromLeftCallback, null);
+ expectedReadCount, readFromLeftCallback, null, false);
} else if (missingOnRight != null) {
AsyncCallbacks.ReadEntriesCallback readFromRightCallback =
new AsyncCallbacks.ReadEntriesCallback() {
@@ -418,7 +419,7 @@ public void readEntriesFailed(ManagedLedgerException exception,
}
};
rangeEntryCache.asyncReadEntry0(lh, missingOnRight.startEntry, missingOnRight.endEntry,
- shouldCacheEntry, readFromRightCallback, null);
+ expectedReadCount, readFromRightCallback, null, false);
}
}
@@ -434,8 +435,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
if (createdByThisThread.get()) {
- CompletableFuture> readResult = rangeEntryCache.readFromStorage(lh, firstEntry,
- lastEntry, shouldCacheEntry);
+ CompletableFuture> readResult = rangeEntryCache.readFromStorage(lh, firstEntry,
+ lastEntry, expectedReadCount);
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..6ac2e7c4a1add
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCache.java
@@ -0,0 +1,267 @@
+/*
+ * 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.CachedEntry;
+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
+public class RangeCache {
+ private final RangeCacheRemovalQueue removalQueue;
+
+ // Map from key to nodes inside the linked list
+ private final ConcurrentNavigableMap entries;
+ private AtomicLong size; // Total size of values stored in cache
+
+ /**
+ * Construct a new RangeCache.
+ */
+ public RangeCache(RangeCacheRemovalQueue removalQueue) {
+ this.removalQueue = removalQueue;
+ this.size = new AtomicLong(0);
+ this.entries = new ConcurrentSkipListMap<>();
+ }
+
+ /**
+ * 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("CachedEntry '" + 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 getValue(key, entries.get(key));
+ }
+
+ private CachedEntry getValue(Position key, RangeCacheEntryWrapper valueWrapper) {
+ if (valueWrapper == null) {
+ return null;
+ } else {
+ CachedEntry value = valueWrapper.getValue(key);
+ if (value == null) {
+ // the wrapper has been recycled and contains another key
+ return null;
+ }
+ try {
+ value.retain();
+ } catch (IllegalReferenceCountException e) {
+ // CachedEntry 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 {
+ // CachedEntry 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 = getValue(entry.getKey(), entry.getValue());
+ 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) {
+ 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 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() {
+ 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..127d6e241b84a
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheEntryWrapper.java
@@ -0,0 +1,123 @@
+/*
+ * 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.CachedEntry;
+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);
+ }
+ }
+
+ CachedEntry getValue(Position key) {
+ 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);
+ }
+ if (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..a1200b6d670e9
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheRemovalQueue.java
@@ -0,0 +1,212 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+import java.util.Queue;
+import org.apache.commons.lang3.tuple.Pair;
+import org.jctools.queues.MpscUnboundedArrayQueue;
+
+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 Queue removalQueue = new MpscUnboundedArrayQueue<>(
+ REMOVAL_QUEUE_CHUNK_SIZE);
+ private final RangeCacheRemovalQueueStash stash = new RangeCacheRemovalQueueStash();
+
+ public Pair evictLEntriesBeforeTimestamp(long timestampNanos) {
+ return evictEntries(
+ (e, c) -> e.timestampNanos < timestampNanos ? EvictionResult.REMOVE : EvictionResult.STASH_AND_STOP,
+ true);
+ }
+
+ public Pair evictLeastAccessedEntries(long sizeToFree) {
+ checkArgument(sizeToFree > 0);
+ return evictEntries(
+ (e, c) -> {
+ if (!e.value.canEvict()) {
+ return EvictionResult.STASH;
+ }
+ return c.removedSize < sizeToFree ? EvictionResult.REMOVE : EvictionResult.STASH_AND_STOP;
+ }, false);
+ }
+
+ public boolean addEntry(RangeCacheEntryWrapper newWrapper) {
+ return removalQueue.offer(newWrapper);
+ }
+
+ class RangeCacheRemovalQueueStash {
+ // TODO: consider using a more efficient data structure, for example, a linked list of lists
+ // and keeping a pool of lists to recycle
+ List entries = new ArrayList<>();
+ int size = 0;
+ int removed = 0;
+
+ public void add(RangeCacheEntryWrapper entry) {
+ entries.add(entry);
+ size++;
+ }
+
+ public boolean evictEntries(EvictionPredicate evictionPredicate, RangeCacheRemovalCounters counters,
+ boolean processAllEntriesInStash) {
+ boolean continueEviction = doEvictEntries(evictionPredicate, counters, processAllEntriesInStash);
+ maybeTrim();
+ return continueEviction;
+ }
+
+ private boolean doEvictEntries(EvictionPredicate evictionPredicate, RangeCacheRemovalCounters counters,
+ boolean processAllEntriesInStash) {
+ for (int i = 0; i < entries.size(); i++) {
+ RangeCacheEntryWrapper entry = entries.get(i);
+ if (entry == null) {
+ continue;
+ }
+ EvictionResult evictionResult = handleEviction(evictionPredicate, entry, counters);
+ if (!evictionResult.shouldStash()) {
+ entries.set(i, null);
+ removed++;
+ }
+ if (!processAllEntriesInStash && (!evictionResult.isContinueEviction() || Thread.currentThread()
+ .isInterrupted())) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ void maybeTrim() {
+ if (removed == size) {
+ entries.clear();
+ size = 0;
+ removed = 0;
+ } else if (size > 1000 && removed > size / 2) {
+ List newEntries = new ArrayList<>(size - removed);
+ for (RangeCacheEntryWrapper entry : entries) {
+ if (entry != null) {
+ newEntries.add(entry);
+ }
+ }
+ entries = newEntries;
+ size = entries.size();
+ removed = 0;
+ }
+ }
+ }
+
+ enum EvictionResult {
+ REMOVE, STASH, STASH_AND_STOP, MISSING;
+
+ boolean isContinueEviction() {
+ return this != STASH_AND_STOP;
+ }
+
+ boolean shouldStash() {
+ return this == STASH || this == STASH_AND_STOP;
+ }
+
+ boolean shouldRecycle() {
+ 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, boolean alwaysProcessAllEntriesInStash) {
+ RangeCacheRemovalCounters counters = RangeCacheRemovalCounters.create();
+ boolean continueEviction = stash.evictEntries(evictionPredicate, counters, alwaysProcessAllEntriesInStash);
+ if (continueEviction) {
+ handleQueue(evictionPredicate, counters);
+ }
+ return handleRemovalResult(counters);
+ }
+
+ private void handleQueue(EvictionPredicate evictionPredicate,
+ RangeCacheRemovalCounters counters) {
+ // peek the first entry in the queue so that we can avoid stashing entries
+ // when eviction should be stopped at the first entry
+ RangeCacheEntryWrapper peekedEntry = removalQueue.peek();
+ if (peekedEntry == null) {
+ return;
+ }
+ EvictionResult peekedEntryEvictionResult = peekedEntry.withWriteLock(e -> {
+ return evaluateEvictionPredicate(evictionPredicate, counters, e);
+ });
+ if (!peekedEntryEvictionResult.isContinueEviction()) {
+ return;
+ }
+ while (!Thread.currentThread().isInterrupted()) {
+ RangeCacheEntryWrapper entry = removalQueue.poll();
+ if (entry == null) {
+ break;
+ }
+ EvictionResult evictionResult = handleEviction(evictionPredicate, entry, counters);
+ if (evictionResult.shouldStash()) {
+ stash.add(entry);
+ }
+ 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;
+ });
+ if (evictionResult.shouldRecycle()) {
+ // recycle the entry after it has been removed from the queue
+ entry.recycle();
+ }
+ 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 cb006a5f0cea9..11f951b73c5f9 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
@@ -22,29 +22,33 @@
import static java.util.Objects.requireNonNull;
import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.PooledByteBufAllocator;
+import java.util.ArrayList;
import java.util.Collection;
-import java.util.Iterator;
+import java.util.Collections;
import java.util.List;
+import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.IntSupplier;
import org.apache.bookkeeper.client.api.BKException;
import org.apache.bookkeeper.client.api.LedgerEntry;
import org.apache.bookkeeper.client.api.ReadHandle;
-import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.mledger.CachedEntry;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.PositionFactory;
+import org.apache.bookkeeper.mledger.impl.CachedEntryImpl;
import org.apache.bookkeeper.mledger.impl.EntryImpl;
+import org.apache.bookkeeper.mledger.impl.EntryReadCountHandlerImpl;
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;
@@ -58,27 +62,28 @@ public class RangeEntryCacheImpl implements EntryCache {
* Overhead per-entry to take into account the envelope.
*/
private static final long BOOKKEEPER_READ_OVERHEAD_PER_ENTRY = 64;
+ private static final int DEFAULT_ESTIMATED_ENTRY_SIZE = 10 * 1024;
+ private static final boolean DEFAULT_CACHE_INDIVIDUAL_READ_ENTRY = false;
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;
- private volatile long estimatedEntrySize = 10 * 1024;
-
- private final long readEntryTimeoutMillis;
-
private static final double MB = 1024 * 1024;
- public RangeEntryCacheImpl(RangeEntryCacheManagerImpl manager, ManagedLedgerImpl ml, boolean copyEntries) {
+ private final LongAdder totalAddedEntriesSize = new LongAdder();
+ private final LongAdder totalAddedEntriesCount = new LongAdder();
+
+ 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.readEntryTimeoutMillis = getManagedLedgerConfig().getReadEntryTimeoutSeconds();
- this.entries = new RangeCache<>(EntryImpl::getLength, EntryImpl::getTimestamp);
+ this.entries = new RangeCache(rangeCacheRemovalQueue);
this.copyEntries = copyEntries;
if (log.isDebugEnabled()) {
@@ -117,49 +122,48 @@ InflightReadsLimiter getPendingReadsLimiter() {
);
@Override
- public boolean insert(EntryImpl entry) {
- if (!manager.hasSpaceInCache()) {
- if (log.isDebugEnabled()) {
- log.debug("[{}] Skipping cache while doing eviction: {} - size: {}", ml.getName(), entry.getPosition(),
- entry.getLength());
- }
- return false;
- }
+ public CachedEntry insert(Entry entry) {
+ int entryLength = entry.getLength();
if (log.isDebugEnabled()) {
log.debug("[{}] Adding entry to cache: {} - size: {}", ml.getName(), entry.getPosition(),
- entry.getLength());
+ entryLength);
}
Position position = entry.getPosition();
if (entries.exists(position)) {
- return false;
+ return null;
}
ByteBuf cachedData;
if (copyEntries) {
cachedData = copyEntry(entry);
if (cachedData == null) {
- return false;
+ return null;
}
} else {
// Use retain here to have the same counter increase as in the copy entry scenario
- cachedData = entry.getDataBuffer().retain();
+ // Use retainedDuplicate to ensure that the readerIndex changes on the original buffer
+ // are not reflected in the cache
+ cachedData = entry.getDataBuffer().retainedDuplicate();
}
- EntryImpl cacheEntry = EntryImpl.create(position, cachedData);
+ CachedEntryImpl cacheEntry =
+ CachedEntryImpl.create(position, cachedData, (EntryReadCountHandlerImpl) entry.getReadCountHandler());
cachedData.release();
if (entries.put(position, cacheEntry)) {
- manager.entryAdded(entry.getLength());
- return true;
+ totalAddedEntriesSize.add(entryLength);
+ totalAddedEntriesCount.increment();
+ manager.entryAdded(entryLength);
+ return cacheEntry;
} else {
// entry was not inserted into cache, we need to discard it
cacheEntry.release();
- return false;
+ return null;
}
}
- 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
@@ -173,10 +177,7 @@ private ByteBuf copyEntry(EntryImpl entry) {
}
if (size > 0) {
- ByteBuf entryBuf = entry.getDataBuffer();
- int readerIdx = entryBuf.readerIndex();
- cachedData.writeBytes(entryBuf);
- entryBuf.readerIndex(readerIdx);
+ cachedData.writeBytes(entry.getDataBuffer().duplicate());
}
return cachedData;
@@ -226,7 +227,23 @@ public void invalidateAllEntries(long ledgerId) {
public void asyncReadEntry(ReadHandle lh, Position position, final ReadEntryCallback callback,
final Object ctx) {
try {
- asyncReadEntry0(lh, position, callback, ctx);
+ asyncReadEntriesByPosition(lh, position, position, 1,
+ () -> DEFAULT_CACHE_INDIVIDUAL_READ_ENTRY ? 1 : 0,
+ new ReadEntriesCallback() {
+ @Override
+ public void readEntriesComplete(List entries, Object ctx) {
+ if (entries.isEmpty()) {
+ callback.readEntryFailed(new ManagedLedgerException("Could not read given position"), ctx);
+ } else {
+ callback.readEntryComplete(entries.get(0), ctx);
+ }
+ }
+
+ @Override
+ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
+ callback.readEntryFailed(exception, ctx);
+ }
+ }, ctx, true);
} catch (Throwable t) {
log.warn("failed to read entries for {}-{}", lh.getId(), position, t);
// invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt
@@ -237,52 +254,11 @@ public void asyncReadEntry(ReadHandle lh, Position position, final ReadEntryCall
}
}
- private void asyncReadEntry0(ReadHandle lh, Position position, final ReadEntryCallback callback,
- final Object ctx) {
- if (log.isDebugEnabled()) {
- log.debug("[{}] Reading entry ledger {}: {}", ml.getName(), lh.getId(), position.getEntryId());
- }
- EntryImpl entry = entries.get(position);
- if (entry != null) {
- EntryImpl cachedEntry = EntryImpl.create(entry);
- entry.release();
- manager.mlFactoryMBean.recordCacheHit(cachedEntry.getLength());
- callback.readEntryComplete(cachedEntry, ctx);
- } else {
- ReadEntryUtils.readAsync(ml, lh, position.getEntryId(), position.getEntryId()).thenAcceptAsync(
- ledgerEntries -> {
- try {
- Iterator iterator = ledgerEntries.iterator();
- if (iterator.hasNext()) {
- LedgerEntry ledgerEntry = iterator.next();
- EntryImpl returnEntry = RangeEntryCacheManagerImpl.create(ledgerEntry, interceptor);
-
- ml.getMbean().recordReadEntriesOpsCacheMisses(1, returnEntry.getLength());
- manager.mlFactoryMBean.recordCacheMiss(1, returnEntry.getLength());
- ml.getMbean().addReadEntriesSample(1, returnEntry.getLength());
- callback.readEntryComplete(returnEntry, ctx);
- } else {
- // got an empty sequence
- callback.readEntryFailed(new ManagedLedgerException("Could not read given position"),
- ctx);
- }
- } finally {
- ledgerEntries.close();
- }
- }, ml.getExecutor()).exceptionally(exception -> {
- ml.invalidateLedgerHandle(lh);
- pendingReadsManager.invalidateLedger(lh.getId());
- callback.readEntryFailed(createManagedLedgerException(exception), ctx);
- return null;
- });
- }
- }
-
@Override
- public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry,
+ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount,
final ReadEntriesCallback callback, Object ctx) {
try {
- asyncReadEntry0(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx);
+ asyncReadEntry0(lh, firstEntry, lastEntry, expectedReadCount, callback, ctx, true);
} catch (Throwable t) {
log.warn("failed to read entries for {}--{}-{}", lh.getId(), firstEntry, lastEntry, t);
// invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt
@@ -294,38 +270,129 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boole
}
@SuppressWarnings({ "unchecked", "rawtypes" })
- void asyncReadEntry0(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry,
- final ReadEntriesCallback callback, Object ctx) {
- asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx, null);
+ void asyncReadEntry0(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount,
+ final ReadEntriesCallback callback, Object ctx, boolean acquirePermits) {
+ final long ledgerId = lh.getId();
+ final int numberOfEntries = (int) (lastEntry - firstEntry) + 1;
+ final Position firstPosition = PositionFactory.create(ledgerId, firstEntry);
+ final Position lastPosition = PositionFactory.create(ledgerId, lastEntry);
+ asyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, callback, ctx,
+ acquirePermits);
}
- void asyncReadEntry0WithLimits(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry,
- final ReadEntriesCallback originalCallback, Object ctx, InflightReadsLimiter.Handle handle) {
+ void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries,
+ IntSupplier expectedReadCount, final ReadEntriesCallback originalCallback,
+ Object ctx, boolean acquirePermits) {
+ checkArgument(firstPosition.getLedgerId() == lastPosition.getLedgerId(),
+ "Invalid range. Entries %s and %s should be in the same ledger.",
+ firstPosition, lastPosition);
+ checkArgument(firstPosition.getLedgerId() == lh.getId(),
+ "Invalid ReadHandle. The ledger %s of the range positions should match the handle's ledger %s.",
+ firstPosition.getLedgerId(), lh.getId());
- final AsyncCallbacks.ReadEntriesCallback callback =
- handlePendingReadsLimits(lh, firstEntry, lastEntry, shouldCacheEntry,
- originalCallback, ctx, handle);
- if (callback == null) {
- return;
+ if (log.isDebugEnabled()) {
+ log.debug("[{}] Reading {} entries in range {} to {}", ml.getName(), numberOfEntries, firstPosition,
+ lastPosition);
}
- final long ledgerId = lh.getId();
- final int entriesToRead = (int) (lastEntry - firstEntry) + 1;
- final Position firstPosition = PositionFactory.create(lh.getId(), firstEntry);
- final Position lastPosition = PositionFactory.create(lh.getId(), lastEntry);
+ InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter();
+ if (!acquirePermits || pendingReadsLimiter.isDisabled()) {
+ doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount,
+ originalCallback, ctx);
+ } else {
+ long estimatedEntrySize = getEstimatedEntrySize();
+ long estimatedReadSize = numberOfEntries * estimatedEntrySize;
+ if (log.isDebugEnabled()) {
+ log.debug("Estimated read size: {} bytes for {} entries with {} estimated entry size",
+ estimatedReadSize,
+ numberOfEntries, estimatedEntrySize);
+ }
+ Optional optionalHandle =
+ pendingReadsLimiter.acquire(estimatedReadSize, handle -> {
+ // permits were not immediately available, callback will be executed when permits are acquired
+ // or timeout
+ ml.getExecutor().execute(() -> {
+ doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries,
+ expectedReadCount, originalCallback, ctx, handle, estimatedReadSize);
+ });
+ });
+ // permits were immediately available and acquired
+ if (optionalHandle.isPresent()) {
+ doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries,
+ expectedReadCount, originalCallback, ctx, optionalHandle.get(), estimatedReadSize);
+ }
+ }
+ }
- if (log.isDebugEnabled()) {
- log.debug("[{}] Reading entries range ledger {}: {} to {}", ml.getName(), ledgerId, firstEntry, lastEntry);
+ void doAsyncReadEntriesWithAcquiredPermits(ReadHandle lh, Position firstPosition, Position lastPosition,
+ int numberOfEntries, IntSupplier expectedReadCount,
+ final ReadEntriesCallback originalCallback, Object ctx,
+ InflightReadsLimiter.Handle handle, long estimatedReadSize) {
+ if (!handle.success) {
+ String message = "Couldn't acquire enough permits "
+ + "on the max reads in flight limiter to read from ledger "
+ + lh.getId()
+ + ", " + getName()
+ + ", estimated read size " + estimatedReadSize + " bytes"
+ + " for " + numberOfEntries
+ + " entries (check managedLedgerMaxReadsInFlightSizeInMB, "
+ + "managedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis and "
+ + "managedLedgerMaxReadsInFlightPermitsAcquireQueueSize)";
+ log.error(message);
+ originalCallback.readEntriesFailed(new ManagedLedgerException.TooManyRequestsException(message), ctx);
+ return;
}
+ InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter();
+ ReadEntriesCallback wrappedCallback = new ReadEntriesCallback() {
+ @Override
+ public void readEntriesComplete(List entries, Object ctx2) {
+ if (!entries.isEmpty()) {
+ // release permits only when entries have been handled
+ AtomicInteger remainingCount = new AtomicInteger(entries.size());
+ for (Entry entry : entries) {
+ ((EntryImpl) entry).onDeallocate(() -> {
+ if (remainingCount.decrementAndGet() <= 0) {
+ pendingReadsLimiter.release(handle);
+ }
+ });
+ }
+ } else {
+ pendingReadsLimiter.release(handle);
+ }
+ originalCallback.readEntriesComplete(entries, ctx2);
+ }
- Collection cachedEntries = entries.getRange(firstPosition, lastPosition);
+ @Override
+ public void readEntriesFailed(ManagedLedgerException exception, Object ctx2) {
+ pendingReadsLimiter.release(handle);
+ originalCallback.readEntriesFailed(exception, ctx2);
+ }
+ };
+ doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount,
+ wrappedCallback, ctx);
+ }
- if (cachedEntries.size() == entriesToRead) {
+ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries,
+ IntSupplier expectedReadCount, final ReadEntriesCallback callback,
+ Object ctx) {
+ Collection cachedEntries;
+ if (firstPosition.compareTo(lastPosition) == 0) {
+ CachedEntry cachedEntry = entries.get(firstPosition);
+ if (cachedEntry == null) {
+ cachedEntries = Collections.emptyList();
+ } else {
+ cachedEntries = Collections.singleton(cachedEntry);
+ }
+ } else {
+ cachedEntries = entries.getRange(firstPosition, lastPosition);
+ }
+
+ if (cachedEntries.size() == numberOfEntries) {
long totalCachedSize = 0;
- final List entriesToReturn = Lists.newArrayListWithExpectedSize(entriesToRead);
+ final List entriesToReturn = new ArrayList<>(numberOfEntries);
// All entries found in cache
- for (EntryImpl entry : cachedEntries) {
+ for (CachedEntry entry : cachedEntries) {
entriesToReturn.add(EntryImpl.create(entry));
totalCachedSize += entry.getLength();
entry.release();
@@ -333,89 +400,34 @@ void asyncReadEntry0WithLimits(ReadHandle lh, long firstEntry, long lastEntry, b
manager.mlFactoryMBean.recordCacheHits(entriesToReturn.size(), totalCachedSize);
if (log.isDebugEnabled()) {
- log.debug("[{}] Ledger {} -- Found in cache entries: {}-{}", ml.getName(), ledgerId, firstEntry,
- lastEntry);
+ log.debug("[{}] Cache hit for {} entries in range {} to {}", ml.getName(), numberOfEntries,
+ firstPosition, lastPosition);
}
-
- callback.readEntriesComplete((List) entriesToReturn, ctx);
-
+ 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());
}
// Read all the entries from bookkeeper
- pendingReadsManager.readEntries(lh, firstEntry, lastEntry,
- shouldCacheEntry, callback, ctx);
-
+ pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(),
+ expectedReadCount, callback, ctx);
}
}
- private AsyncCallbacks.ReadEntriesCallback handlePendingReadsLimits(ReadHandle lh,
- long firstEntry, long lastEntry,
- boolean shouldCacheEntry,
- AsyncCallbacks.ReadEntriesCallback originalCallback,
- Object ctx, InflightReadsLimiter.Handle handle) {
- InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter();
- if (pendingReadsLimiter.isDisabled()) {
- return originalCallback;
+ private long getEstimatedEntrySize() {
+ long estimatedEntrySize = getAvgEntrySize();
+ if (estimatedEntrySize == 0) {
+ estimatedEntrySize = DEFAULT_ESTIMATED_ENTRY_SIZE;
}
- long estimatedReadSize = (1 + lastEntry - firstEntry)
- * (estimatedEntrySize + BOOKKEEPER_READ_OVERHEAD_PER_ENTRY);
- final AsyncCallbacks.ReadEntriesCallback callback;
- InflightReadsLimiter.Handle newHandle = pendingReadsLimiter.acquire(estimatedReadSize, handle);
- if (!newHandle.success) {
- long now = System.currentTimeMillis();
- if (now - newHandle.creationTime > readEntryTimeoutMillis) {
- String message = "Time-out elapsed while acquiring enough permits "
- + "on the memory limiter to read from ledger "
- + lh.getId()
- + ", " + getName()
- + ", estimated read size " + estimatedReadSize + " bytes"
- + " for " + (1 + lastEntry - firstEntry)
- + " entries (check managedLedgerMaxReadsInFlightSizeInMB)";
- log.error(message);
- pendingReadsLimiter.release(newHandle);
- originalCallback.readEntriesFailed(
- new ManagedLedgerException.TooManyRequestsException(message), ctx);
- return null;
- }
- ml.getExecutor().execute(() -> {
- asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry,
- originalCallback, ctx, newHandle);
- });
- return null;
- } else {
- callback = new AsyncCallbacks.ReadEntriesCallback() {
-
- @Override
- public void readEntriesComplete(List entries, Object ctx) {
- if (!entries.isEmpty()) {
- long size = entries.get(0).getLength();
- estimatedEntrySize = size;
-
- AtomicInteger remainingCount = new AtomicInteger(entries.size());
- for (Entry entry : entries) {
- ((EntryImpl) entry).onDeallocate(() -> {
- if (remainingCount.decrementAndGet() <= 0) {
- pendingReadsLimiter.release(newHandle);
- }
- });
- }
- } else {
- pendingReadsLimiter.release(newHandle);
- }
- originalCallback.readEntriesComplete(entries, ctx);
- }
+ return estimatedEntrySize + BOOKKEEPER_READ_OVERHEAD_PER_ENTRY;
+ }
- @Override
- public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
- pendingReadsLimiter.release(newHandle);
- originalCallback.readEntriesFailed(exception, ctx);
- }
- };
- }
- return callback;
+ private long getAvgEntrySize() {
+ long totalAddedEntriesCount = this.totalAddedEntriesCount.sum();
+ long totalAddedEntriesSize = this.totalAddedEntriesSize.sum();
+ return totalAddedEntriesCount != 0 ? totalAddedEntriesSize / totalAddedEntriesCount : 0;
}
/**
@@ -423,13 +435,14 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
* @param lh the handle
* @param firstEntry the first entry
* @param lastEntry the last entry
- * @param shouldCacheEntry if we should put the entry into the cache
+ * @param expectedReadCount 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,
+ IntSupplier expectedReadCount) {
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());
@@ -438,17 +451,16 @@ CompletableFuture> readFromStorage(ReadHandle lh,
try {
// We got the entries, we need to transform them to a List<> type
long totalSize = 0;
- final List entriesToReturn =
- Lists.newArrayListWithExpectedSize(entriesToRead);
+ final List entriesToReturn = new ArrayList<>(entriesToRead);
+ int expectedReadCountValue = expectedReadCount.getAsInt();
for (LedgerEntry e : ledgerEntries) {
- EntryImpl entry = RangeEntryCacheManagerImpl.create(e, interceptor);
+ EntryImpl entry =
+ RangeEntryCacheManagerImpl.create(e, interceptor, expectedReadCountValue);
entriesToReturn.add(entry);
- totalSize += entry.getLength();
- if (shouldCacheEntry) {
- EntryImpl cacheEntry = EntryImpl.create(entry);
- insert(cacheEntry);
- cacheEntry.release();
+ if (expectedReadCountValue > 0) {
+ insert(entry);
}
+ totalSize += entry.getLength();
}
ml.getMbean().recordReadEntriesOpsCacheMisses(entriesToReturn.size(), totalSize);
@@ -485,32 +497,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..f07831f47a42a
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerEvictionHandler.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 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
+public class RangeEntryCacheManagerEvictionHandler implements EntryCachesEvictionHandler {
+
+ private final RangeEntryCacheManagerImpl manager;
+ private final RangeCacheRemovalQueue rangeCacheRemovalQueue;
+
+ public RangeEntryCacheManagerEvictionHandler(RangeEntryCacheManagerImpl manager,
+ RangeCacheRemovalQueue rangeCacheRemovalQueue) {
+ this.manager = manager;
+ this.rangeCacheRemovalQueue = rangeCacheRemovalQueue;
+ }
+
+ @Override
+ public void invalidateEntriesBeforeTimestampNanos(long timestamp) {
+ Pair evictedPair = rangeCacheRemovalQueue.evictLEntriesBeforeTimestamp(timestamp);
+ manager.entriesRemoved(evictedPair.getRight(), evictedPair.getLeft());
+ }
+
+ @Override
+ 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 34be25df1f476..9be3fc3774a22 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,17 +18,20 @@
*/
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 java.util.concurrent.atomic.AtomicReference;
import org.apache.bookkeeper.client.api.LedgerEntry;
import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
+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;
@@ -45,9 +48,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;
@@ -57,15 +61,20 @@ public class RangeEntryCacheManagerImpl implements EntryCacheManager {
private static final double evictionTriggerThresholdPercent = 0.98;
- public RangeEntryCacheManagerImpl(ManagedLedgerFactoryImpl factory, OpenTelemetry openTelemetry) {
- this.maxSize = factory.getConfig().getMaxCacheSize();
- this.inflightReadsLimiter = new InflightReadsLimiter(
- factory.getConfig().getManagedLedgerMaxReadsInFlightSize(), openTelemetry);
+ public RangeEntryCacheManagerImpl(ManagedLedgerFactoryImpl factory, OrderedScheduler scheduledExecutor,
+ OpenTelemetry openTelemetry) {
+ ManagedLedgerFactoryConfig config = factory.getConfig();
+ this.maxSize = config.getMaxCacheSize();
+ this.inflightReadsLimiter = new InflightReadsLimiter(config.getManagedLedgerMaxReadsInFlightSize(),
+ config.getManagedLedgerMaxReadsInFlightPermitsAcquireQueueSize(),
+ config.getManagedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis(),
+ scheduledExecutor, openTelemetry);
this.evictionTriggerThreshold = (long) (maxSize * evictionTriggerThresholdPercent);
- this.cacheEvictionWatermark = factory.getConfig().getCacheEvictionWatermark();
- this.evictionPolicy = new EntryCacheDefaultEvictionPolicy();
+ this.cacheEvictionWatermark = config.getCacheEvictionWatermark();
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);
}
@@ -76,7 +85,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;
@@ -111,40 +121,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) {
@@ -167,6 +209,25 @@ public double getCacheEvictionWatermark() {
return cacheEvictionWatermark;
}
+ @Override
+ public EntryCachesEvictionHandler getEvictionHandler() {
+ return evictionHandler;
+ }
+
+ @Override
+ 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);
+ }
+ }
+
@Override
public void clear() {
caches.values().forEach(EntryCache::clear);
@@ -176,7 +237,8 @@ public static Entry create(long ledgerId, long entryId, ByteBuf data) {
return EntryImpl.create(ledgerId, entryId, data);
}
- public static EntryImpl create(LedgerEntry ledgerEntry, ManagedLedgerInterceptor interceptor) {
+ public static EntryImpl create(LedgerEntry ledgerEntry, ManagedLedgerInterceptor interceptor,
+ int expectedReadCount) {
ManagedLedgerInterceptor.PayloadProcessorHandle processorHandle = null;
if (interceptor != null) {
ByteBuf duplicateBuffer = ledgerEntry.getEntryBuffer().retainedDuplicate();
@@ -189,7 +251,7 @@ public static EntryImpl create(LedgerEntry ledgerEntry, ManagedLedgerInterceptor
duplicateBuffer.release();
}
}
- EntryImpl returnEntry = EntryImpl.create(ledgerEntry);
+ EntryImpl returnEntry = EntryImpl.create(ledgerEntry, expectedReadCount);
if (processorHandle != null) {
processorHandle.release();
ledgerEntry.close();
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/AbstractCASReferenceCounted.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/AbstractCASReferenceCounted.java
index 8b0e25f1348ca..94ed4089adac0 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/AbstractCASReferenceCounted.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/AbstractCASReferenceCounted.java
@@ -98,6 +98,7 @@ private boolean release0(int decrement) {
}
if (refCntUpdater.compareAndSet(this, refCnt, refCnt - decrement)) {
+ refCountDecremented(refCnt - decrement, decrement);
if (refCnt == decrement) {
deallocate();
return true;
@@ -107,6 +108,10 @@ private boolean release0(int decrement) {
}
}
+ protected void refCountDecremented(int refCount, int decrement) {
+ // no-op
+ }
+
/**
* Called once {@link #refCnt()} is equals 0.
*/
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 2f2b161a30684..0000000000000
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java
+++ /dev/null
@@ -1,496 +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;
- }
-
- V getValue(K key) {
- 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);
- }
- if (localKey != 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 getValue(key, entries.get(key));
- }
-
- private Value getValue(Key key, EntryWrapper valueWrapper) {
- if (valueWrapper == null) {
- return null;
- } else {
- Value value = valueWrapper.getValue(key);
- 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 = getValue(entry.getKey(), entry.getValue());
- 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) {
- RemovalCounters counters = RemovalCounters.create();
- Map> subMap = entries.subMap(first, true, last, lastInclusive);
- for (Map.Entry> entry : subMap.entrySet()) {
- removeEntry(entry, counters, true);
- }
- return handleRemovalResult(counters);
- }
-
- enum RemoveEntryResult {
- ENTRY_REMOVED,
- CONTINUE_LOOP,
- BREAK_LOOP;
- }
-
- private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters,
- boolean skipInvalid) {
- return removeEntry(entry, counters, skipInvalid, x -> true);
- }
-
- private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters,
- boolean skipInvalid, Predicate removeCondition) {
- Key key = entry.getKey();
- EntryWrapper entryWrapper = entry.getValue();
- Value value = entryWrapper.getValue(key);
- if (value == null) {
- // the wrapper has already been recycled and contains another key
- if (!skipInvalid) {
- EntryWrapper removed = entries.remove(key);
- if (removed != null) {
- // log and remove the entry without releasing the value
- log.info("Key {} does not match the entry's value wrapper's key {}, removed entry by key without "
- + "releasing the value", key, entryWrapper.getKey());
- counters.entryRemoved(removed.getSize());
- return RemoveEntryResult.ENTRY_REMOVED;
- }
- }
- return RemoveEntryResult.CONTINUE_LOOP;
- }
- try {
- // add extra retain to avoid value being released while we are removing it
- value.retain();
- } catch (IllegalReferenceCountException e) {
- // Value was already released
- if (!skipInvalid) {
- // remove the specific entry without releasing the value
- if (entries.remove(key, entryWrapper)) {
- log.info("Value was already released for key {}, removed entry without releasing the value", key);
- counters.entryRemoved(entryWrapper.getSize());
- return RemoveEntryResult.ENTRY_REMOVED;
- }
- }
- return RemoveEntryResult.CONTINUE_LOOP;
- }
- if (!value.matchesKey(key)) {
- // this is unexpected since the IdentityWrapper.getValue(key) already checked that the value matches the key
- log.warn("Unexpected race condition. Value {} does not match the key {}. Removing entry.", value, key);
- }
- try {
- if (!removeCondition.test(value)) {
- return RemoveEntryResult.BREAK_LOOP;
- }
- if (!skipInvalid) {
- // 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);
- }
- }
- } else if (skipInvalid && value.refCnt() > 1 && entries.remove(key, entryWrapper)) {
- // when skipInvalid is true, we don't remove the entry if it doesn't match matches the key
- // or the refCnt is invalid
- counters.entryRemoved(entryWrapper.getSize());
- entryWrapper.recycle();
- // remove the cache reference
- value.release();
- }
- } finally {
- // remove the extra retain
- value.release();
- }
- return RemoveEntryResult.ENTRY_REMOVED;
- }
-
- 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) {
- 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, false);
- }
- return handleRemovalResult(counters);
- }
-
- /**
- *
- * @param maxTimestamp the max timestamp of the entries to be evicted
- * @return the tota
- */
- public Pair evictLEntriesBeforeTimestamp(long maxTimestamp) {
- RemovalCounters counters = RemovalCounters.create();
- while (!Thread.currentThread().isInterrupted()) {
- Map.Entry> entry = entries.firstEntry();
- if (entry == null) {
- break;
- }
- if (removeEntry(entry, counters, false, 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() {
- RemovalCounters counters = RemovalCounters.create();
- while (!Thread.currentThread().isInterrupted()) {
- Map.Entry> entry = entries.firstEntry();
- if (entry == null) {
- break;
- }
- removeEntry(entry, counters, false);
- }
- 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/main/java/org/apache/bookkeeper/mledger/util/StatsBuckets.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/StatsBuckets.java
index 60c0a7f6c9d22..c9dca1fc958a3 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/StatsBuckets.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/StatsBuckets.java
@@ -97,7 +97,7 @@ public long getSum() {
}
public double getAvg() {
- return sum / (double) count;
+ return count != 0 ? sum / (double) count : Double.NaN;
}
public void addAll(StatsBuckets other) {
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..329ea705a6563 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,18 @@
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.assertNotNull;
+import static org.testng.Assert.assertNull;
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;
@@ -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);
}
@@ -153,13 +154,13 @@ public void doubleInsert() throws Exception {
EntryCacheManager cacheManager = factory2.getEntryCacheManager();
EntryCache cache1 = cacheManager.getEntryCache(ml1);
- assertTrue(cache1.insert(EntryImpl.create(1, 1, new byte[4])));
- assertTrue(cache1.insert(EntryImpl.create(1, 0, new byte[3])));
+ assertNotNull(cache1.insert(EntryImpl.create(1, 1, new byte[4])));
+ assertNotNull(cache1.insert(EntryImpl.create(1, 0, new byte[3])));
assertEquals(cache1.getSize(), 7);
assertEquals(cacheManager.getSize(), 7);
- assertFalse(cache1.insert(EntryImpl.create(1, 0, new byte[5])));
+ assertNull(cache1.insert(EntryImpl.create(1, 0, new byte[5])));
assertEquals(cache1.getSize(), 7);
assertEquals(cacheManager.getSize(), 7);
@@ -184,7 +185,7 @@ public void cacheSizeUpdate() throws Exception {
// Put entries into cache.
for (int i = 0; i < 20; i++) {
entries.add(EntryImpl.create(1, i, new byte[i + 1]));
- assertTrue(cache1.insert(entries.get(i)));
+ assertNotNull(cache1.insert(entries.get(i)));
}
assertEquals(210, cacheManager.getSize());
@@ -280,15 +281,19 @@ 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.setCacheEvictionByExpectedReadCount(false);
+ 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 +332,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 +345,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/EntryCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java
index 551aa80bc07dc..d3f24ea93a550 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java
@@ -27,7 +27,6 @@
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
-
import io.netty.buffer.Unpooled;
import java.util.ArrayList;
import java.util.List;
@@ -35,6 +34,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.function.Consumer;
+import java.util.function.IntSupplier;
import lombok.Cleanup;
import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException;
import org.apache.bookkeeper.client.api.LedgerEntries;
@@ -82,7 +82,7 @@ public void testRead() throws Exception {
}
when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9));
- final var entries = readEntry(entryCache, lh, 0, 9, false, null);
+ final var entries = readEntry(entryCache, lh, 0, 9, () -> 0, null);
assertEquals(entries.size(), 10);
entries.forEach(Entry::release);
@@ -106,7 +106,7 @@ public void testReadMissingBefore() throws Exception {
}
when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9));
- final var entries = readEntry(entryCache, lh, 0, 9, false, null);
+ final var entries = readEntry(entryCache, lh, 0, 9, () -> 0, null);
assertEquals(entries.size(), 10);
}
@@ -125,7 +125,7 @@ public void testReadMissingAfter() throws Exception {
}
when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9));
- final var entries = readEntry(entryCache, lh, 0, 9, false, null);
+ final var entries = readEntry(entryCache, lh, 0, 9, () -> 0, null);
assertEquals(entries.size(), 10);
}
@@ -145,7 +145,7 @@ public void testReadMissingMiddle() throws Exception {
entryCache.insert(EntryImpl.create(0, 9, data));
when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9));
- final var entries = readEntry(entryCache, lh, 0, 9, false, null);
+ final var entries = readEntry(entryCache, lh, 0, 9, () -> 0, null);
assertEquals(entries.size(), 10);
}
@@ -165,7 +165,7 @@ public void testReadMissingMultiple() throws Exception {
entryCache.insert(EntryImpl.create(0, 8, data));
when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9));
- final var entries = readEntry(entryCache, lh, 0, 9, false, null);
+ final var entries = readEntry(entryCache, lh, 0, 9, () -> 0, null);
assertEquals(entries.size(), 10);
}
@@ -178,25 +178,25 @@ public void testCachedReadReturnsDifferentByteBuffer() throws Exception {
@Cleanup(value = "clear")
EntryCache entryCache = cacheManager.getEntryCache(ml);
- readEntry(entryCache, lh, 0, 1, true, e -> {
+ readEntry(entryCache, lh, 0, 1, () -> 1, e -> {
assertTrue(e instanceof ManagedLedgerException);
assertTrue(e.getMessage().contains("LastConfirmedEntry is null when reading ledger 0"));
});
when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(-1, -1));
- readEntry(entryCache, lh, 0, 1, true, e -> {
+ readEntry(entryCache, lh, 0, 1, () -> 1, e -> {
assertTrue(e instanceof ManagedLedgerException);
assertTrue(e.getMessage().contains("LastConfirmedEntry is -1:-1 when reading ledger 0"));
});
when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 0));
- readEntry(entryCache, lh, 0, 1, true, e -> {
+ readEntry(entryCache, lh, 0, 1, () -> 1, e -> {
assertTrue(e instanceof ManagedLedgerException);
assertTrue(e.getMessage().contains("LastConfirmedEntry is 0:0 when reading entry 1"));
});
when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 1));
- List cacheMissEntries = readEntry(entryCache, lh, 0, 1, true, null);
+ List cacheMissEntries = readEntry(entryCache, lh, 0, 1, () -> 1, null);
// Ensure first entry is 0 and
assertEquals(cacheMissEntries.size(), 2);
assertEquals(cacheMissEntries.get(0).getEntryId(), 0);
@@ -205,7 +205,7 @@ public void testCachedReadReturnsDifferentByteBuffer() throws Exception {
// Move the reader index to simulate consumption
cacheMissEntries.get(0).getDataBuffer().readerIndex(10);
- List cacheHitEntries = readEntry(entryCache, lh, 0, 1, true, null);
+ List cacheHitEntries = readEntry(entryCache, lh, 0, 1, () -> 1, null);
assertEquals(cacheHitEntries.get(0).getEntryId(), 0);
assertEquals(cacheHitEntries.get(0).getDataBuffer().readerIndex(), 0);
}
@@ -229,7 +229,7 @@ public void testReadWithError() throws Exception {
entryCache.insert(EntryImpl.create(0, 2, data));
when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9));
- readEntry(entryCache, lh, 0, 9, false, e ->
+ readEntry(entryCache, lh, 0, 9, () -> 0, e ->
assertTrue(e instanceof ManagedLedgerException.LedgerNotExistException));
}
@@ -253,10 +253,10 @@ static ReadHandle getLedgerHandle() {
}
private List readEntry(EntryCache entryCache, ReadHandle lh, long firstEntry, long lastEntry,
- boolean shouldCacheEntry, Consumer assertion)
+ IntSupplier expectedReadCount, Consumer assertion)
throws InterruptedException {
final var future = new CompletableFuture>();
- entryCache.asyncReadEntry(lh, firstEntry, lastEntry, shouldCacheEntry, new ReadEntriesCallback() {
+ entryCache.asyncReadEntry(lh, firstEntry, lastEntry, expectedReadCount, new ReadEntriesCallback() {
@Override
public void readEntriesComplete(List entries, Object ctx) {
future.complete(entries);
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..9b7b765d35b93 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,39 @@ 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);
+ }
+
+ @Test
+ public void testCountNumberOfCursorsAtSamePositionOrBefore() {
+ ManagedCursorContainer container = new ManagedCursorContainer();
+ List cursors = IntStream.rangeClosed(1, 1000)
+ .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()));
+ for (int i = 1; i <= 1000; i++) {
+ ManagedCursor cursor = container.get("cursor" + i);
+ int numberOfCursorsBefore = container.getNumberOfCursorsAtSamePositionOrBefore(cursor);
+ assertThat(numberOfCursorsBefore).describedAs("cursor:%s", cursor).isEqualTo(i);
+ }
+ }
}
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 67c928c8a5617..d7aec4dc33c9b 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
@@ -433,6 +433,7 @@ public void shouldKeepEntriesInCacheByEarliestReadPosition() throws ManagedLedge
// This test case reproduces issue #16054
ManagedLedgerConfig config = new ManagedLedgerConfig();
+ initManagedLedgerConfig(config);
factory.updateCacheEvictionTimeThreshold(TimeUnit.MILLISECONDS
.toNanos(30000));
@@ -488,6 +489,8 @@ public void shouldKeepEntriesInCacheByEarliestMarkDeletePosition() throws Manage
// This test case reproduces issue #16054
ManagedLedgerConfig config = new ManagedLedgerConfig();
+ initManagedLedgerConfig(config);
+ initManagedLedgerConfig(config);
config.setCacheEvictionByMarkDeletedPosition(true);
factory.updateCacheEvictionTimeThreshold(TimeUnit.MILLISECONDS
.toNanos(30000));
@@ -549,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() {
@@ -632,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);
@@ -693,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);
@@ -812,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) {
@@ -841,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;
@@ -1011,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");
@@ -1023,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");
@@ -1043,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");
@@ -1078,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);
@@ -1103,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");
@@ -1136,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");
@@ -1167,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");
@@ -1286,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));
@@ -1362,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));
@@ -1567,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++) {
@@ -1675,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);
@@ -1699,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();
@@ -1710,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());
@@ -1738,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");
@@ -1775,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;
@@ -1798,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());
}
@@ -1825,7 +1844,9 @@ public void invalidateConsumedEntriesFromCache() throws Exception {
@Test
public void invalidateEntriesFromCacheByMarkDeletePosition() throws Exception {
ManagedLedgerConfig config = new ManagedLedgerConfig();
+ initManagedLedgerConfig(config);
config.setCacheEvictionByMarkDeletedPosition(true);
+ config.setCacheEvictionByExpectedReadCount(false);
ManagedLedgerImpl ledger =
(ManagedLedgerImpl) factory.open("my_test_ledger_for_invalidateEntriesFromCacheByMarkDeletePosition",
config);
@@ -1849,28 +1870,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());
}
@@ -2095,6 +2130,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");
@@ -2116,6 +2152,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);
@@ -2137,6 +2174,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);
@@ -2159,6 +2197,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);
@@ -2187,6 +2226,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);
@@ -2218,6 +2258,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);
@@ -2243,6 +2284,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
@@ -2271,6 +2313,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);
@@ -2300,6 +2343,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);
@@ -2336,6 +2380,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);
@@ -2366,6 +2411,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);
@@ -2388,6 +2434,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);
@@ -2422,6 +2469,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);
@@ -2454,6 +2502,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);
@@ -2498,6 +2547,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);
@@ -2566,6 +2616,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);
@@ -2636,6 +2687,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);
@@ -2694,6 +2746,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");
@@ -2732,9 +2785,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
@@ -2772,8 +2828,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
@@ -2783,6 +2838,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));
@@ -2797,6 +2854,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));
@@ -2804,6 +2863,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);
@@ -2893,6 +2954,7 @@ public void testLazyRecoverCursor() throws Exception {
}, 10, TimeUnit.SECONDS);
ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig();
+ initManagedLedgerConfig(managedLedgerConfig);
managedLedgerConfig.setLazyCursorRecovery(true);
Long startLedgerRecovery = System.currentTimeMillis();
@@ -3045,7 +3107,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;
@@ -3079,14 +3141,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);
@@ -3100,7 +3162,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);
@@ -3130,7 +3192,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);
BookKeeper bk = mock(BookKeeper.class);
@@ -3139,11 +3201,12 @@ public void testManagedLedgerWithReadEntryTimeOut() throws Exception {
String ctxStr = "timeoutCtx";
CompletableFuture entriesFuture = new CompletableFuture<>();
ReadHandle ledgerHandle = mock(ReadHandle.class);
- doReturn(entriesFuture).when(ledgerHandle).readAsync(PositionFactory.EARLIEST.getLedgerId(),
- PositionFactory.EARLIEST.getEntryId());
+ Position readPosition = PositionFactory.create(0, 0);
+ doReturn(entriesFuture).when(ledgerHandle).readAsync(readPosition.getLedgerId(),
+ readPosition.getEntryId());
// (1) test read-timeout for: ManagedLedger.asyncReadEntry(..)
- ledger.asyncReadEntry(ledgerHandle, PositionFactory.EARLIEST, new ReadEntryCallback() {
+ ledger.asyncReadEntry(ledgerHandle, readPosition, new ReadEntryCallback() {
@Override
public void readEntryComplete(Entry entry, Object ctx) {
responseException1.set(null);
@@ -3163,9 +3226,8 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
// (2) test read-timeout for: ManagedLedger.asyncReadEntry(..)
AtomicReference responseException2 = new AtomicReference<>();
- Position readPositionRef = PositionFactory.EARLIEST;
ManagedCursorImpl cursor = new ManagedCursorImpl(bk, ledger, "cursor1");
- OpReadEntry opReadEntry = OpReadEntry.create(cursor, readPositionRef, 1, new ReadEntriesCallback() {
+ OpReadEntry opReadEntry = OpReadEntry.create(cursor, readPosition, 1, new ReadEntriesCallback() {
@Override
public void readEntriesComplete(List entries, Object ctx) {
@@ -3178,7 +3240,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
}
}, null, PositionFactory.LATEST, null);
- ledger.asyncReadEntry(ledgerHandle, PositionFactory.EARLIEST.getEntryId(), PositionFactory.EARLIEST.getEntryId(),
+ ledger.asyncReadEntry(ledgerHandle, readPosition.getEntryId(), readPosition.getEntryId(),
opReadEntry, ctxStr);
retryStrategically((test) -> {
return responseException2.get() != null;
@@ -3193,7 +3255,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();
@@ -3248,7 +3310,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);
@@ -3450,7 +3512,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);
@@ -3485,7 +3547,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
@@ -3497,6 +3559,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);
@@ -3528,6 +3591,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);
@@ -3567,6 +3631,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);
@@ -3582,6 +3647,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));
@@ -3602,6 +3668,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);
@@ -3645,6 +3712,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);
@@ -3670,6 +3738,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");
@@ -3713,6 +3782,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.
@@ -3747,6 +3817,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));
@@ -3779,6 +3850,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
@@ -3825,6 +3897,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);
@@ -3945,6 +4018,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");
@@ -3983,6 +4057,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");
@@ -4008,6 +4083,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);
@@ -4029,6 +4105,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);
@@ -4066,7 +4143,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");
@@ -4172,6 +4249,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);
@@ -4218,6 +4296,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));
@@ -4245,6 +4324,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.
@@ -4345,6 +4425,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/InflightReadsLimiterTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java
index 68135598e3339..725e9bcbbc407 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java
@@ -21,22 +21,30 @@
import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat;
import static org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization.FREE;
import static org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization.USED;
+import static org.mockito.Mockito.mock;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertTrue;
import io.opentelemetry.api.OpenTelemetry;
import io.opentelemetry.sdk.OpenTelemetrySdk;
import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk;
import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader;
import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.commons.lang3.tuple.Pair;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
@Slf4j
public class InflightReadsLimiterTest {
+ private static final int ACQUIRE_QUEUE_SIZE = 1000;
+ private static final int ACQUIRE_TIMEOUT_MILLIS = 500;
@DataProvider
private static Object[][] isDisabled() {
@@ -53,7 +61,8 @@ public void testDisabled(long maxReadsInFlightSize, boolean shouldBeDisabled) th
@Cleanup var openTelemetry = otel.getLeft();
@Cleanup var metricReader = otel.getRight();
- var limiter = new InflightReadsLimiter(maxReadsInFlightSize, openTelemetry);
+ var limiter = new InflightReadsLimiter(maxReadsInFlightSize, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ mock(ScheduledExecutorService.class), openTelemetry);
assertEquals(limiter.isDisabled(), shouldBeDisabled);
if (shouldBeDisabled) {
@@ -72,15 +81,18 @@ public void testBasicAcquireRelease() throws Exception {
@Cleanup var openTelemetry = otel.getLeft();
@Cleanup var metricReader = otel.getRight();
- InflightReadsLimiter limiter = new InflightReadsLimiter(100, openTelemetry);
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ mock(ScheduledExecutorService.class), openTelemetry);
assertEquals(100, limiter.getRemainingBytes());
assertLimiterMetrics(metricReader, 100, 0, 100);
- InflightReadsLimiter.Handle handle = limiter.acquire(100, null);
+ Optional optionalHandle = limiter.acquire(100, null);
assertEquals(0, limiter.getRemainingBytes());
+ assertTrue(optionalHandle.isPresent());
+ InflightReadsLimiter.Handle handle = optionalHandle.get();
assertTrue(handle.success);
- assertEquals(handle.acquiredPermits, 100);
- assertEquals(1, handle.trials);
+ assertEquals(handle.permits, 100);
assertLimiterMetrics(metricReader, 100, 100, 0);
limiter.release(handle);
@@ -91,117 +103,48 @@ public void testBasicAcquireRelease() throws Exception {
@Test
public void testNotEnoughPermits() throws Exception {
- InflightReadsLimiter limiter = new InflightReadsLimiter(100, OpenTelemetry.noop());
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ mock(ScheduledExecutorService.class), OpenTelemetry.noop());
assertEquals(100, limiter.getRemainingBytes());
- InflightReadsLimiter.Handle handle = limiter.acquire(100, null);
+ Optional optionalHandle = limiter.acquire(100, null);
assertEquals(0, limiter.getRemainingBytes());
+ assertTrue(optionalHandle.isPresent());
+ InflightReadsLimiter.Handle handle = optionalHandle.get();
assertTrue(handle.success);
- assertEquals(handle.acquiredPermits, 100);
- assertEquals(1, handle.trials);
+ assertEquals(handle.permits, 100);
- InflightReadsLimiter.Handle handle2 = limiter.acquire(100, null);
+ MutableObject handle2Reference = new MutableObject<>();
+ Optional optionalHandle2 = limiter.acquire(100, handle2Reference::setValue);
assertEquals(0, limiter.getRemainingBytes());
- assertFalse(handle2.success);
- assertEquals(handle2.acquiredPermits, 0);
- assertEquals(1, handle2.trials);
+ assertFalse(optionalHandle2.isPresent());
limiter.release(handle);
- assertEquals(100, limiter.getRemainingBytes());
-
- handle2 = limiter.acquire(100, handle2);
- assertEquals(0, limiter.getRemainingBytes());
- assertTrue(handle2.success);
- assertEquals(handle2.acquiredPermits, 100);
- assertEquals(2, handle2.trials);
-
- limiter.release(handle2);
- assertEquals(100, limiter.getRemainingBytes());
-
- }
-
- @Test
- public void testPartialAcquire() throws Exception {
- InflightReadsLimiter limiter = new InflightReadsLimiter(100, OpenTelemetry.noop());
- assertEquals(100, limiter.getRemainingBytes());
-
- InflightReadsLimiter.Handle handle = limiter.acquire(30, null);
- assertEquals(70, limiter.getRemainingBytes());
- assertTrue(handle.success);
- assertEquals(handle.acquiredPermits, 30);
- assertEquals(1, handle.trials);
-
- InflightReadsLimiter.Handle handle2 = limiter.acquire(100, null);
- assertEquals(0, limiter.getRemainingBytes());
- assertFalse(handle2.success);
- assertEquals(handle2.acquiredPermits, 70);
- assertEquals(1, handle2.trials);
-
- limiter.release(handle);
-
- handle2 = limiter.acquire(100, handle2);
- assertEquals(0, limiter.getRemainingBytes());
- assertTrue(handle2.success);
- assertEquals(handle2.acquiredPermits, 100);
- assertEquals(2, handle2.trials);
+ assertNotNull(handle2Reference.getValue());
+ assertTrue(handle2Reference.getValue().success);
- limiter.release(handle2);
+ limiter.release(handle2Reference.getValue());
assertEquals(100, limiter.getRemainingBytes());
-
}
@Test
- public void testTooManyTrials() throws Exception {
- InflightReadsLimiter limiter = new InflightReadsLimiter(100, OpenTelemetry.noop());
+ public void testAcquireTimeout() throws Exception {
+ @Cleanup("shutdownNow")
+ ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ executor, OpenTelemetry.noop());
assertEquals(100, limiter.getRemainingBytes());
+ limiter.acquire(100, null);
- InflightReadsLimiter.Handle handle = limiter.acquire(30, null);
- assertEquals(70, limiter.getRemainingBytes());
- assertTrue(handle.success);
- assertEquals(handle.acquiredPermits, 30);
- assertEquals(1, handle.trials);
-
- InflightReadsLimiter.Handle handle2 = limiter.acquire(100, null);
- assertEquals(0, limiter.getRemainingBytes());
- assertFalse(handle2.success);
- assertEquals(handle2.acquiredPermits, 70);
- assertEquals(1, handle2.trials);
-
- handle2 = limiter.acquire(100, handle2);
- assertEquals(0, limiter.getRemainingBytes());
- assertFalse(handle2.success);
- assertEquals(handle2.acquiredPermits, 70);
- assertEquals(2, handle2.trials);
-
- handle2 = limiter.acquire(100, handle2);
- assertEquals(0, limiter.getRemainingBytes());
- assertFalse(handle2.success);
- assertEquals(handle2.acquiredPermits, 70);
- assertEquals(3, handle2.trials);
+ MutableObject handle2Reference = new MutableObject<>();
+ Optional optionalHandle2 = limiter.acquire(100, handle2Reference::setValue);
+ assertFalse(optionalHandle2.isPresent());
- handle2 = limiter.acquire(100, handle2);
- assertEquals(0, limiter.getRemainingBytes());
- assertFalse(handle2.success);
- assertEquals(handle2.acquiredPermits, 70);
- assertEquals(4, handle2.trials);
-
- // too many trials, start from scratch
- handle2 = limiter.acquire(100, handle2);
- assertEquals(70, limiter.getRemainingBytes());
- assertFalse(handle2.success);
- assertEquals(handle2.acquiredPermits, 0);
- assertEquals(1, handle2.trials);
-
- limiter.release(handle);
-
- handle2 = limiter.acquire(100, handle2);
- assertEquals(0, limiter.getRemainingBytes());
- assertTrue(handle2.success);
- assertEquals(handle2.acquiredPermits, 100);
- assertEquals(2, handle2.trials);
-
- limiter.release(handle2);
- assertEquals(100, limiter.getRemainingBytes());
+ Thread.sleep(ACQUIRE_TIMEOUT_MILLIS + 100);
+ assertNotNull(handle2Reference.getValue());
+ assertFalse(handle2Reference.getValue().success);
}
private Pair buildOpenTelemetryAndReader() {
diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java
index 01976f648aba4..a7296f188020d 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java
@@ -35,6 +35,8 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.IntSupplier;
import java.util.stream.Collectors;
import lombok.Data;
import lombok.extern.slf4j.Slf4j;
@@ -91,7 +93,8 @@ void setupMocks() {
config.setReadEntryTimeoutSeconds(10000);
when(rangeEntryCache.getName()).thenReturn("my-topic");
when(rangeEntryCache.getManagedLedgerConfig()).thenReturn(config);
- inflighReadsLimiter = new InflightReadsLimiter(0, OpenTelemetry.noop());
+ inflighReadsLimiter = new InflightReadsLimiter(0, 0, 0,
+ mock(ScheduledExecutorService.class), OpenTelemetry.noop());
when(rangeEntryCache.getPendingReadsLimiter()).thenReturn(inflighReadsLimiter);
pendingReadsManager = new PendingReadsManager(rangeEntryCache);
doAnswer(new Answer() {
@@ -101,14 +104,14 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
ReadHandle rh = invocationOnMock.getArgument(0);
long startEntry = invocationOnMock.getArgument(1);
long endEntry = invocationOnMock.getArgument(2);
- boolean shouldCacheEntry = invocationOnMock.getArgument(3);
+ IntSupplier expectedReadCount = invocationOnMock.getArgument(3);
AsyncCallbacks.ReadEntriesCallback callback = invocationOnMock.getArgument(4);
Object ctx = invocationOnMock.getArgument(5);
- pendingReadsManager.readEntries(lh, startEntry, endEntry, shouldCacheEntry, callback, ctx);
+ pendingReadsManager.readEntries(lh, startEntry, endEntry, expectedReadCount, callback, ctx);
return null;
}
}).when(rangeEntryCache).asyncReadEntry0(any(), anyLong(), anyLong(),
- anyBoolean(), any(), any());
+ any(), any(), any(), anyBoolean());
lh = mock(ReadHandle.class);
ml = mock(ManagedLedgerImpl.class);
@@ -164,17 +167,17 @@ private void verifyRange(List entries, long firstEntry, long endEntry)
private static class PreparedReadFromStorage extends CompletableFuture> {
final long firstEntry;
final long endEntry;
- final boolean shouldCacheEntry;
+ final IntSupplier expectedReadCount;
- public PreparedReadFromStorage(long firstEntry, long endEntry, boolean shouldCacheEntry) {
+ public PreparedReadFromStorage(long firstEntry, long endEntry, IntSupplier expectedReadCount) {
this.firstEntry = firstEntry;
this.endEntry = endEntry;
- this.shouldCacheEntry = shouldCacheEntry;
+ this.expectedReadCount = expectedReadCount;
}
@Override
public String toString() {
- return "PreparedReadFromStorage("+firstEntry+","+endEntry+","+shouldCacheEntry+")";
+ return "PreparedReadFromStorage("+firstEntry+","+endEntry+","+expectedReadCount+")";
}
public void storageReadCompleted() {
@@ -183,12 +186,13 @@ public void storageReadCompleted() {
}
private PreparedReadFromStorage prepareReadFromStorage(ReadHandle lh, RangeEntryCacheImpl rangeEntryCache,
- long firstEntry, long endEntry, boolean shouldCacheEntry) {
- PreparedReadFromStorage read = new PreparedReadFromStorage(firstEntry, endEntry, shouldCacheEntry);
- log.info("prepareReadFromStorage from {} to {} shouldCacheEntry {}", firstEntry, endEntry, shouldCacheEntry);
- when(rangeEntryCache.readFromStorage(eq(lh), eq(firstEntry), eq(endEntry), eq(shouldCacheEntry))).thenAnswer(
+ long firstEntry, long endEntry,
+ IntSupplier expectedReadCount) {
+ PreparedReadFromStorage read = new PreparedReadFromStorage(firstEntry, endEntry, expectedReadCount);
+ log.info("prepareReadFromStorage from {} to {} expectedReadCount {}", firstEntry, endEntry, expectedReadCount);
+ when(rangeEntryCache.readFromStorage(eq(lh), eq(firstEntry), eq(endEntry), eq(expectedReadCount))).thenAnswer(
(invocationOnMock -> {
- log.info("readFromStorage from {} to {} shouldCacheEntry {}", firstEntry, endEntry, shouldCacheEntry);
+ log.info("readFromStorage from {} to {} expectedReadCount {}", firstEntry, endEntry, expectedReadCount);
return read;
})
);
@@ -200,13 +204,13 @@ public void simpleRead() throws Exception {
long firstEntry = 100;
long endEntry = 199;
- boolean shouldCacheEntry = false;
+ IntSupplier expectedReadCount = () -> 0;
PreparedReadFromStorage read1
- = prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, shouldCacheEntry);
+ = prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount);
CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntry, endEntry, shouldCacheEntry, callback, CTX);
+ pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX);
// complete the read
read1.storageReadCompleted();
@@ -225,16 +229,16 @@ public void simpleConcurrentReadPerfectMatch() throws Exception {
long firstEntry = 100;
long endEntry = 199;
- boolean shouldCacheEntry = false;
+ IntSupplier expectedReadCount = () -> 0;
- PreparedReadFromStorage read1 = prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, shouldCacheEntry);
+ PreparedReadFromStorage read1 = prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount);
PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache);
CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntry, endEntry, shouldCacheEntry, callback, CTX);
+ pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX);
CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntry, endEntry, shouldCacheEntry, callback2, CTX2);
+ pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback2, CTX2);
// complete the read from BK
// only one read completes 2 callbacks
@@ -267,17 +271,17 @@ public void simpleConcurrentReadIncluding() throws Exception {
long firstEntrySecondRead = firstEntry + 10;
long endEntrySecondRead = endEntry - 10;
- boolean shouldCacheEntry = false;
+ IntSupplier expectedReadCount = () -> 0;
- PreparedReadFromStorage read1 = prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, shouldCacheEntry);
+ PreparedReadFromStorage read1 = prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount);
PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache);
CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntry, endEntry, shouldCacheEntry, callback, CTX);
+ pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX);
CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, shouldCacheEntry, callback2, CTX2);
+ pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, CTX2);
// complete the read from BK
// only one read completes 2 callbacks
@@ -313,20 +317,20 @@ public void simpleConcurrentReadMissingLeft() throws Exception {
long firstEntrySecondRead = firstEntry - 10;
long endEntrySecondRead = endEntry;
- boolean shouldCacheEntry = false;
+ IntSupplier expectedReadCount = () -> 0;
PreparedReadFromStorage read1 =
- prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, shouldCacheEntry);
+ prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount);
PreparedReadFromStorage readForLeft =
- prepareReadFromStorage(lh, rangeEntryCache, firstEntrySecondRead, firstEntry - 1, shouldCacheEntry);
+ prepareReadFromStorage(lh, rangeEntryCache, firstEntrySecondRead, firstEntry - 1, expectedReadCount);
PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache);
CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntry, endEntry, shouldCacheEntry, callback, CTX);
+ pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX);
CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, shouldCacheEntry, callback2, CTX2);
+ pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, CTX2);
// complete the read from BK
read1.storageReadCompleted();
@@ -353,20 +357,20 @@ public void simpleConcurrentReadMissingRight() throws Exception {
long firstEntrySecondRead = firstEntry;
long endEntrySecondRead = endEntry + 10;
- boolean shouldCacheEntry = false;
+ IntSupplier expectedReadCount = () -> 0;
PreparedReadFromStorage read1 =
- prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, shouldCacheEntry);
+ prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount);
PreparedReadFromStorage readForRight =
- prepareReadFromStorage(lh, rangeEntryCache, endEntry + 1, endEntrySecondRead, shouldCacheEntry);
+ prepareReadFromStorage(lh, rangeEntryCache, endEntry + 1, endEntrySecondRead, expectedReadCount);
PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache);
CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntry, endEntry, shouldCacheEntry, callback, CTX);
+ pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX);
CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, shouldCacheEntry, callback2, CTX2);
+ pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, CTX2);
// complete the read from BK
read1.storageReadCompleted();
@@ -393,23 +397,23 @@ public void simpleConcurrentReadMissingBoth() throws Exception {
long firstEntrySecondRead = firstEntry - 10;
long endEntrySecondRead = endEntry + 10;
- boolean shouldCacheEntry = false;
+ IntSupplier expectedReadCount = () -> 0;
PreparedReadFromStorage read1 =
- prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, shouldCacheEntry);
+ prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount);
PreparedReadFromStorage readForLeft =
- prepareReadFromStorage(lh, rangeEntryCache, firstEntrySecondRead, firstEntry - 1, shouldCacheEntry);
+ prepareReadFromStorage(lh, rangeEntryCache, firstEntrySecondRead, firstEntry - 1, expectedReadCount);
PreparedReadFromStorage readForRight =
- prepareReadFromStorage(lh, rangeEntryCache, endEntry + 1, endEntrySecondRead, shouldCacheEntry);
+ prepareReadFromStorage(lh, rangeEntryCache, endEntry + 1, endEntrySecondRead, expectedReadCount);
PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache);
CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntry, endEntry, shouldCacheEntry, callback, CTX);
+ pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX);
CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, shouldCacheEntry, callback2, CTX2);
+ pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, CTX2);
// complete the read from BK
read1.storageReadCompleted();
@@ -437,20 +441,20 @@ public void simpleConcurrentReadNoMatch() throws Exception {
long firstEntrySecondRead = 1000;
long endEntrySecondRead = 1099;
- boolean shouldCacheEntry = false;
+ IntSupplier expectedReadCount = () -> 0;
PreparedReadFromStorage read1 =
- prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, shouldCacheEntry);
+ prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount);
PreparedReadFromStorage read2 =
- prepareReadFromStorage(lh, rangeEntryCache, firstEntrySecondRead, endEntrySecondRead, shouldCacheEntry);
+ prepareReadFromStorage(lh, rangeEntryCache, firstEntrySecondRead, endEntrySecondRead, expectedReadCount);
PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache);
CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntry, endEntry, shouldCacheEntry, callback, CTX);
+ pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX);
CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback();
- pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, shouldCacheEntry, callback2, CTX2);
+ pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, CTX2);
read1.storageReadCompleted();
callback.get();
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..78b7e6858ede0
--- /dev/null
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeCacheTest.java
@@ -0,0 +1,355 @@
+/*
+ * 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.assertNull;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+import io.netty.buffer.Unpooled;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.Cleanup;
+import org.apache.bookkeeper.mledger.CachedEntry;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.PositionFactory;
+import org.apache.bookkeeper.mledger.impl.CachedEntryImpl;
+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.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()), null);
+ }
+
+ private static Position createPosition(int i) {
+ return PositionFactory.create(0, i);
+ }
+
+
+ @Test
+ public void evictLEntriesBeforeTimestamp() {
+ 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);
+
+ Pair evictedSize = removalQueue.evictLEntriesBeforeTimestamp(timestamp);
+ assertEquals(evictedSize.getRight().longValue(), 6);
+ assertEquals(evictedSize.getLeft().longValue(), 3);
+
+ assertEquals(cache.getSize(), 4);
+ assertEquals(cache.getNumberOfEntries(), 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);
+ }
+}
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 4bcf2cc6c4e35..0000000000000
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java
+++ /dev/null
@@ -1,341 +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.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.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import lombok.Cleanup;
-import lombok.Data;
-import org.apache.commons.lang3.tuple.Pair;
-import org.awaitility.Awaitility;
-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);
- }
-
-
- @Test
- public void customTimeExtraction() {
- 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);
-
- Pair evictedSize = cache.evictLEntriesBeforeTimestamp(3);
- assertEquals(evictedSize.getRight().longValue(), 6);
- assertEquals(evictedSize.getLeft().longValue(), 3);
-
- assertEquals(cache.getSize(), 4);
- assertEquals(cache.getNumberOfEntries(), 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