diff --git a/.github/workflows/ci-maven-cache-update.yaml b/.github/workflows/ci-maven-cache-update.yaml
index d37794c2e54a1..e3451c87429d2 100644
--- a/.github/workflows/ci-maven-cache-update.yaml
+++ b/.github/workflows/ci-maven-cache-update.yaml
@@ -49,7 +49,7 @@ jobs:
name: Update Maven dependency cache for ${{ matrix.name }}
env:
JOB_NAME: Update Maven dependency cache for ${{ matrix.name }}
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
runs-on: ${{ matrix.runs-on }}
timeout-minutes: 45
diff --git a/.github/workflows/ci-owasp-dependency-check.yaml b/.github/workflows/ci-owasp-dependency-check.yaml
index a1c6dd594d3a2..4a2e12a06377a 100644
--- a/.github/workflows/ci-owasp-dependency-check.yaml
+++ b/.github/workflows/ci-owasp-dependency-check.yaml
@@ -24,7 +24,7 @@ on:
workflow_dispatch:
env:
- MAVEN_OPTS: -Xss1500k -Xmx1500m -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000
+ MAVEN_OPTS: -Xss1500k -Xmx2048m -XX:+UnlockDiagnosticVMOptions -XX:GCLockerRetryAllocationCount=100 -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000
JDK_DISTRIBUTION: corretto
NIST_NVD_API_KEY: ${{ secrets.NIST_NVD_API_KEY }}
@@ -34,7 +34,7 @@ jobs:
name: Check ${{ matrix.branch }}
env:
JOB_NAME: Check ${{ matrix.branch }}
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
runs-on: ubuntu-22.04
timeout-minutes: 75
strategy:
diff --git a/.github/workflows/pulsar-ci-flaky.yaml b/.github/workflows/pulsar-ci-flaky.yaml
index 4731381857d2f..89529386f31ee 100644
--- a/.github/workflows/pulsar-ci-flaky.yaml
+++ b/.github/workflows/pulsar-ci-flaky.yaml
@@ -37,7 +37,7 @@ concurrency:
cancel-in-progress: true
env:
- MAVEN_OPTS: -Xss1500k -Xmx1500m -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000
+ MAVEN_OPTS: -Xss1500k -Xmx2048m -XX:+UnlockDiagnosticVMOptions -XX:GCLockerRetryAllocationCount=100 -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000
# defines the retention period for the intermediate build artifacts needed for rerunning a failed build job
# it's possible to rerun individual failed jobs when the build artifacts are available
# if the artifacts have already been expired, the complete workflow can be rerun by closing and reopening the PR or by rebasing the PR
@@ -95,7 +95,7 @@ jobs:
env:
JOB_NAME: Flaky tests suite
COLLECT_COVERAGE: "${{ needs.preconditions.outputs.collect_coverage }}"
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
runs-on: ubuntu-22.04
timeout-minutes: 100
if: ${{ needs.preconditions.outputs.docs_only != 'true' }}
diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml
index 494ac771b0420..9c1986c5b5b43 100644
--- a/.github/workflows/pulsar-ci.yaml
+++ b/.github/workflows/pulsar-ci.yaml
@@ -37,7 +37,7 @@ concurrency:
cancel-in-progress: true
env:
- MAVEN_OPTS: -Xss1500k -Xmx1500m -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000
+ MAVEN_OPTS: -Xss1500k -Xmx2048m -XX:+UnlockDiagnosticVMOptions -XX:GCLockerRetryAllocationCount=100 -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000
# defines the retention period for the intermediate build artifacts needed for rerunning a failed build job
# it's possible to rerun individual failed jobs when the build artifacts are available
# if the artifacts have already been expired, the complete workflow can be rerun by closing and reopening the PR or by rebasing the PR
@@ -96,7 +96,7 @@ jobs:
name: Build and License check
env:
JOB_NAME: Build and License check
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
runs-on: ubuntu-22.04
timeout-minutes: 60
if: ${{ needs.preconditions.outputs.docs_only != 'true' }}
@@ -172,7 +172,7 @@ jobs:
env:
JOB_NAME: CI - Unit - ${{ matrix.name }}
COLLECT_COVERAGE: "${{ needs.preconditions.outputs.collect_coverage }}"
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
runs-on: ubuntu-22.04
timeout-minutes: ${{ matrix.timeout || 60 }}
needs: ['preconditions', 'build-and-license-check']
@@ -391,7 +391,7 @@ jobs:
needs: ['preconditions', 'build-and-license-check']
if: ${{ needs.preconditions.outputs.docs_only != 'true'}}
env:
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
steps:
- name: checkout
uses: actions/checkout@v4
@@ -465,7 +465,7 @@ jobs:
env:
JOB_NAME: CI - Integration - ${{ matrix.name }}
PULSAR_TEST_IMAGE_NAME: apachepulsar/java-test-image:latest
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
strategy:
fail-fast: false
matrix:
@@ -731,7 +731,7 @@ jobs:
needs: ['preconditions', 'build-and-license-check']
if: ${{ needs.preconditions.outputs.docs_only != 'true' }}
env:
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
steps:
- name: checkout
uses: actions/checkout@v4
@@ -841,7 +841,7 @@ jobs:
env:
JOB_NAME: CI - System - ${{ matrix.name }}
PULSAR_TEST_IMAGE_NAME: apachepulsar/pulsar-test-latest-version:latest
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
strategy:
fail-fast: false
matrix:
@@ -1072,7 +1072,7 @@ jobs:
env:
JOB_NAME: CI Flaky - System - ${{ matrix.name }}
PULSAR_TEST_IMAGE_NAME: apachepulsar/pulsar-test-latest-version:latest
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
strategy:
fail-fast: false
matrix:
@@ -1214,7 +1214,7 @@ jobs:
needs: ['preconditions', 'integration-tests']
if: ${{ needs.preconditions.outputs.docs_only != 'true' }}
env:
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
steps:
- name: checkout
uses: actions/checkout@v4
@@ -1249,7 +1249,7 @@ jobs:
needs: [ 'preconditions', 'integration-tests' ]
if: ${{ needs.preconditions.outputs.need_owasp == 'true' }}
env:
- DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }}
+ DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
NIST_NVD_API_KEY: ${{ secrets.NIST_NVD_API_KEY }}
steps:
- name: checkout
diff --git a/.mvn/develocity.xml b/.mvn/develocity.xml
index 5c0fbb47c7217..399b37899985d 100644
--- a/.mvn/develocity.xml
+++ b/.mvn/develocity.xml
@@ -24,16 +24,16 @@
#{(env['GRADLE_ENTERPRISE_ACCESS_KEY']?.trim() > '' or env['DEVELOCITY_ACCESS_KEY']?.trim() > '') and !(env['GITHUB_HEAD_REF']?.matches('(?i).*(experiment|wip|private).*') or env['GITHUB_REPOSITORY']?.matches('(?i).*(experiment|wip|private).*'))}
+ pulsar
- https://ge.apache.org
+ https://develocity.apache.org
false
-
- true
- true
-
#{isFalse(env['GITHUB_ACTIONS'])}
+
+ authenticated
+
#{{'0.0.0.0'}}
diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml
index eb998dc3471b8..8ceede33b9cdc 100644
--- a/.mvn/extensions.xml
+++ b/.mvn/extensions.xml
@@ -24,11 +24,11 @@
com.gradle
develocity-maven-extension
- 1.21.6
+ 1.22.2
com.gradle
common-custom-user-data-maven-extension
- 2.0
+ 2.0.1
diff --git a/bin/bookkeeper b/bin/bookkeeper
index 0cc07dd49aba5..445ddb22f9366 100755
--- a/bin/bookkeeper
+++ b/bin/bookkeeper
@@ -183,7 +183,7 @@ fi
OPTS="-cp $BOOKIE_CLASSPATH $OPTS"
# Disable ipv6 as it can cause issues
-OPTS="$OPTS -Djava.net.preferIPv4Stack=true"
+OPTS="-Djava.net.preferIPv4Stack=true $OPTS"
OPTS="$OPTS $BOOKIE_MEM $BOOKIE_GC $BOOKIE_GC_LOG $BOOKIE_EXTRA_OPTS"
diff --git a/bin/function-localrunner b/bin/function-localrunner
index 2e0aa0f6dffe2..39ce218ceaac2 100755
--- a/bin/function-localrunner
+++ b/bin/function-localrunner
@@ -103,7 +103,7 @@ fi
# Ensure we can read bigger content from ZK. (It might be
# rarely needed when trying to list many z-nodes under a
# directory)
-OPTS="$OPTS -Djute.maxbuffer=10485760 -Djava.net.preferIPv4Stack=true"
+OPTS="-Djava.net.preferIPv4Stack=true $OPTS -Djute.maxbuffer=10485760"
OPTS="-cp $PULSAR_CLASSPATH $OPTS"
diff --git a/bin/pulsar b/bin/pulsar
index 20ed1f7f22b0f..f201829cf7259 100755
--- a/bin/pulsar
+++ b/bin/pulsar
@@ -285,7 +285,7 @@ OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`"
# Ensure we can read bigger content from ZK. (It might be
# rarely needed when trying to list many z-nodes under a
# directory)
-OPTS="$OPTS -Djute.maxbuffer=10485760 -Djava.net.preferIPv4Stack=true"
+OPTS="-Djava.net.preferIPv4Stack=true $OPTS -Djute.maxbuffer=10485760"
# Enable TCP keepalive for all Zookeeper client connections
OPTS="$OPTS -Dzookeeper.clientTcpKeepAlive=true"
@@ -321,8 +321,6 @@ else
fi
PULSAR_LOG_APPENDER=${PULSAR_LOG_APPENDER:-"RoutingAppender"}
-PULSAR_LOG_LEVEL=${PULSAR_LOG_LEVEL:-"info"}
-PULSAR_LOG_ROOT_LEVEL=${PULSAR_LOG_ROOT_LEVEL:-"${PULSAR_LOG_LEVEL}"}
PULSAR_ROUTING_APPENDER_DEFAULT=${PULSAR_ROUTING_APPENDER_DEFAULT:-"Console"}
if [ ! -d "$PULSAR_LOG_DIR" ]; then
mkdir -p "$PULSAR_LOG_DIR"
@@ -332,8 +330,14 @@ PULSAR_LOG_IMMEDIATE_FLUSH="${PULSAR_LOG_IMMEDIATE_FLUSH:-"false"}"
#Configure log configuration system properties
OPTS="$OPTS -Dpulsar.log.appender=$PULSAR_LOG_APPENDER"
OPTS="$OPTS -Dpulsar.log.dir=$PULSAR_LOG_DIR"
-OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL"
-OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_ROOT_LEVEL"
+if [ -n "$PULSAR_LOG_LEVEL" ]; then
+ OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL"
+fi
+if [ -n "$PULSAR_LOG_ROOT_LEVEL" ]; then
+ OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_ROOT_LEVEL"
+elif [ -n "$PULSAR_LOG_LEVEL" ]; then
+ OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_LEVEL"
+fi
OPTS="$OPTS -Dpulsar.log.immediateFlush=$PULSAR_LOG_IMMEDIATE_FLUSH"
OPTS="$OPTS -Dpulsar.routing.appender.default=$PULSAR_ROUTING_APPENDER_DEFAULT"
# Configure log4j2 to disable servlet webapp detection so that Garbage free logging can be used
diff --git a/bin/pulsar-admin-common.cmd b/bin/pulsar-admin-common.cmd
index c52bc1389f68a..fea512cfca32c 100644
--- a/bin/pulsar-admin-common.cmd
+++ b/bin/pulsar-admin-common.cmd
@@ -48,7 +48,7 @@ for %%a in ("%PULSAR_LOG_CONF%") do SET "PULSAR_LOG_CONF_BASENAME=%%~nxa"
set "PULSAR_CLASSPATH=%PULSAR_CLASSPATH%;%PULSAR_LOG_CONF_DIR%"
set "OPTS=%OPTS% -Dlog4j.configurationFile="%PULSAR_LOG_CONF_BASENAME%""
-set "OPTS=%OPTS% -Djava.net.preferIPv4Stack=true"
+set "OPTS=-Djava.net.preferIPv4Stack=true %OPTS%"
set "isjava8=false"
FOR /F "tokens=*" %%g IN ('"java -version 2>&1"') do (
@@ -68,15 +68,19 @@ set "OPTS=%OPTS% %PULSAR_EXTRA_OPTS%"
if "%PULSAR_LOG_DIR%" == "" set "PULSAR_LOG_DIR=%PULSAR_HOME%\logs"
if "%PULSAR_LOG_APPENDER%" == "" set "PULSAR_LOG_APPENDER=RoutingAppender"
-if "%PULSAR_LOG_LEVEL%" == "" set "PULSAR_LOG_LEVEL=info"
-if "%PULSAR_LOG_ROOT_LEVEL%" == "" set "PULSAR_LOG_ROOT_LEVEL=%PULSAR_LOG_LEVEL%"
if "%PULSAR_ROUTING_APPENDER_DEFAULT%" == "" set "PULSAR_ROUTING_APPENDER_DEFAULT=Console"
if "%PULSAR_LOG_IMMEDIATE_FLUSH%" == "" set "PULSAR_LOG_IMMEDIATE_FLUSH=false"
set "OPTS=%OPTS% -Dpulsar.log.appender=%PULSAR_LOG_APPENDER%"
set "OPTS=%OPTS% -Dpulsar.log.dir=%PULSAR_LOG_DIR%"
-set "OPTS=%OPTS% -Dpulsar.log.level=%PULSAR_LOG_LEVEL%"
-set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_ROOT_LEVEL%"
+if not "%PULSAR_LOG_LEVEL%" == "" set "OPTS=%OPTS% -Dpulsar.log.level=%PULSAR_LOG_LEVEL%"
+if not "%PULSAR_LOG_ROOT_LEVEL%" == "" (
+ set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_ROOT_LEVEL%"
+) else (
+ if not "%PULSAR_LOG_LEVEL%" == "" (
+ set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_LEVEL%"
+ )
+)
set "OPTS=%OPTS% -Dpulsar.log.immediateFlush=%PULSAR_LOG_IMMEDIATE_FLUSH%"
set "OPTS=%OPTS% -Dpulsar.routing.appender.default=%PULSAR_ROUTING_APPENDER_DEFAULT%"
diff --git a/bin/pulsar-admin-common.sh b/bin/pulsar-admin-common.sh
index 8aa21c00f634d..1b242c6d4b782 100755
--- a/bin/pulsar-admin-common.sh
+++ b/bin/pulsar-admin-common.sh
@@ -89,7 +89,7 @@ fi
PULSAR_CLASSPATH="$PULSAR_JAR:$PULSAR_CLASSPATH:$PULSAR_EXTRA_CLASSPATH"
PULSAR_CLASSPATH="`dirname $PULSAR_LOG_CONF`:$PULSAR_CLASSPATH"
OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`"
-OPTS="$OPTS -Djava.net.preferIPv4Stack=true"
+OPTS="-Djava.net.preferIPv4Stack=true $OPTS"
IS_JAVA_8=$( $JAVA -version 2>&1 | grep version | grep '"1\.8' )
# Start --add-opens options
diff --git a/bin/pulsar-perf b/bin/pulsar-perf
index bdc1dc1ed8b8c..5f07203641d4f 100755
--- a/bin/pulsar-perf
+++ b/bin/pulsar-perf
@@ -132,7 +132,7 @@ fi
PULSAR_CLASSPATH="$PULSAR_JAR:$PULSAR_CLASSPATH:$PULSAR_EXTRA_CLASSPATH"
PULSAR_CLASSPATH="`dirname $PULSAR_LOG_CONF`:$PULSAR_CLASSPATH"
-OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF` -Djava.net.preferIPv4Stack=true"
+OPTS="-Djava.net.preferIPv4Stack=true $OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`"
IS_JAVA_8=$( $JAVA -version 2>&1 | grep version | grep '"1\.8' )
# Start --add-opens options
@@ -147,14 +147,18 @@ OPTS="$OPTS $PULSAR_EXTRA_OPTS"
# log directory & file
PULSAR_LOG_APPENDER=${PULSAR_LOG_APPENDER:-"Console"}
PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-perftest.log"}
-PULSAR_LOG_LEVEL=${PULSAR_LOG_LEVEL:-"info"}
-PULSAR_LOG_ROOT_LEVEL=${PULSAR_LOG_ROOT_LEVEL:-"${PULSAR_LOG_LEVEL}"}
PULSAR_LOG_IMMEDIATE_FLUSH="${PULSAR_LOG_IMMEDIATE_FLUSH:-"false"}"
#Configure log configuration system properties
OPTS="$OPTS -Dpulsar.log.appender=$PULSAR_LOG_APPENDER"
-OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL"
-OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_ROOT_LEVEL"
+if [ -n "$PULSAR_LOG_LEVEL" ]; then
+ OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL"
+fi
+if [ -n "$PULSAR_LOG_ROOT_LEVEL" ]; then
+ OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_ROOT_LEVEL"
+elif [ -n "$PULSAR_LOG_LEVEL" ]; then
+ OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_LEVEL"
+fi
OPTS="$OPTS -Dpulsar.log.immediateFlush=$PULSAR_LOG_IMMEDIATE_FLUSH"
OPTS="$OPTS -Dpulsar.log.dir=$PULSAR_LOG_DIR"
OPTS="$OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE"
diff --git a/buildtools/pom.xml b/buildtools/pom.xml
index d5d81bae041c9..6a9f451b5acb0 100644
--- a/buildtools/pom.xml
+++ b/buildtools/pom.xml
@@ -47,7 +47,7 @@
4.1
10.14.2
3.1.2
- 4.1.116.Final
+ 4.1.117.Final
4.2.3
32.1.1-jre
1.10.12
diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt
index c7278edbb0d30..c6c1b57bd0a2d 100644
--- a/distribution/server/src/assemble/LICENSE.bin.txt
+++ b/distribution/server/src/assemble/LICENSE.bin.txt
@@ -289,26 +289,26 @@ The Apache Software License, Version 2.0
- org.apache.commons-commons-lang3-3.11.jar
- org.apache.commons-commons-text-1.10.0.jar
* Netty
- - io.netty-netty-buffer-4.1.116.Final.jar
- - io.netty-netty-codec-4.1.116.Final.jar
- - io.netty-netty-codec-dns-4.1.116.Final.jar
- - io.netty-netty-codec-http-4.1.116.Final.jar
- - io.netty-netty-codec-http2-4.1.116.Final.jar
- - io.netty-netty-codec-socks-4.1.116.Final.jar
- - io.netty-netty-codec-haproxy-4.1.116.Final.jar
- - io.netty-netty-common-4.1.116.Final.jar
- - io.netty-netty-handler-4.1.116.Final.jar
- - io.netty-netty-handler-proxy-4.1.116.Final.jar
- - io.netty-netty-resolver-4.1.116.Final.jar
- - io.netty-netty-resolver-dns-4.1.116.Final.jar
- - io.netty-netty-resolver-dns-classes-macos-4.1.116.Final.jar
- - io.netty-netty-resolver-dns-native-macos-4.1.116.Final-osx-aarch_64.jar
- - io.netty-netty-resolver-dns-native-macos-4.1.116.Final-osx-x86_64.jar
- - io.netty-netty-transport-4.1.116.Final.jar
- - io.netty-netty-transport-classes-epoll-4.1.116.Final.jar
- - io.netty-netty-transport-native-epoll-4.1.116.Final-linux-aarch_64.jar
- - io.netty-netty-transport-native-epoll-4.1.116.Final-linux-x86_64.jar
- - io.netty-netty-transport-native-unix-common-4.1.116.Final.jar
+ - io.netty-netty-buffer-4.1.117.Final.jar
+ - io.netty-netty-codec-4.1.117.Final.jar
+ - io.netty-netty-codec-dns-4.1.117.Final.jar
+ - io.netty-netty-codec-http-4.1.117.Final.jar
+ - io.netty-netty-codec-http2-4.1.117.Final.jar
+ - io.netty-netty-codec-socks-4.1.117.Final.jar
+ - io.netty-netty-codec-haproxy-4.1.117.Final.jar
+ - io.netty-netty-common-4.1.117.Final.jar
+ - io.netty-netty-handler-4.1.117.Final.jar
+ - io.netty-netty-handler-proxy-4.1.117.Final.jar
+ - io.netty-netty-resolver-4.1.117.Final.jar
+ - io.netty-netty-resolver-dns-4.1.117.Final.jar
+ - io.netty-netty-resolver-dns-classes-macos-4.1.117.Final.jar
+ - io.netty-netty-resolver-dns-native-macos-4.1.117.Final-osx-aarch_64.jar
+ - io.netty-netty-resolver-dns-native-macos-4.1.117.Final-osx-x86_64.jar
+ - io.netty-netty-transport-4.1.117.Final.jar
+ - io.netty-netty-transport-classes-epoll-4.1.117.Final.jar
+ - io.netty-netty-transport-native-epoll-4.1.117.Final-linux-aarch_64.jar
+ - io.netty-netty-transport-native-epoll-4.1.117.Final-linux-x86_64.jar
+ - io.netty-netty-transport-native-unix-common-4.1.117.Final.jar
- io.netty-netty-tcnative-boringssl-static-2.0.69.Final.jar
- io.netty-netty-tcnative-boringssl-static-2.0.69.Final-linux-aarch_64.jar
- io.netty-netty-tcnative-boringssl-static-2.0.69.Final-linux-x86_64.jar
diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt
index 5d2f01779e98b..7bac43c0705f3 100644
--- a/distribution/shell/src/assemble/LICENSE.bin.txt
+++ b/distribution/shell/src/assemble/LICENSE.bin.txt
@@ -344,22 +344,22 @@ The Apache Software License, Version 2.0
- commons-text-1.10.0.jar
- commons-compress-1.26.0.jar
* Netty
- - netty-buffer-4.1.116.Final.jar
- - netty-codec-4.1.116.Final.jar
- - netty-codec-dns-4.1.116.Final.jar
- - netty-codec-http-4.1.116.Final.jar
- - netty-codec-socks-4.1.116.Final.jar
- - netty-codec-haproxy-4.1.116.Final.jar
- - netty-common-4.1.116.Final.jar
- - netty-handler-4.1.116.Final.jar
- - netty-handler-proxy-4.1.116.Final.jar
- - netty-resolver-4.1.116.Final.jar
- - netty-resolver-dns-4.1.116.Final.jar
- - netty-transport-4.1.116.Final.jar
- - netty-transport-classes-epoll-4.1.116.Final.jar
- - netty-transport-native-epoll-4.1.116.Final-linux-aarch_64.jar
- - netty-transport-native-epoll-4.1.116.Final-linux-x86_64.jar
- - netty-transport-native-unix-common-4.1.116.Final.jar
+ - netty-buffer-4.1.117.Final.jar
+ - netty-codec-4.1.117.Final.jar
+ - netty-codec-dns-4.1.117.Final.jar
+ - netty-codec-http-4.1.117.Final.jar
+ - netty-codec-socks-4.1.117.Final.jar
+ - netty-codec-haproxy-4.1.117.Final.jar
+ - netty-common-4.1.117.Final.jar
+ - netty-handler-4.1.117.Final.jar
+ - netty-handler-proxy-4.1.117.Final.jar
+ - netty-resolver-4.1.117.Final.jar
+ - netty-resolver-dns-4.1.117.Final.jar
+ - netty-transport-4.1.117.Final.jar
+ - netty-transport-classes-epoll-4.1.117.Final.jar
+ - netty-transport-native-epoll-4.1.117.Final-linux-aarch_64.jar
+ - netty-transport-native-epoll-4.1.117.Final-linux-x86_64.jar
+ - netty-transport-native-unix-common-4.1.117.Final.jar
- netty-tcnative-boringssl-static-2.0.69.Final.jar
- netty-tcnative-boringssl-static-2.0.69.Final-linux-aarch_64.jar
- netty-tcnative-boringssl-static-2.0.69.Final-linux-x86_64.jar
@@ -370,9 +370,9 @@ The Apache Software License, Version 2.0
- netty-incubator-transport-classes-io_uring-0.0.26.Final.jar
- netty-incubator-transport-native-io_uring-0.0.26.Final-linux-aarch_64.jar
- netty-incubator-transport-native-io_uring-0.0.26.Final-linux-x86_64.jar
- - netty-resolver-dns-classes-macos-4.1.116.Final.jar
- - netty-resolver-dns-native-macos-4.1.116.Final-osx-aarch_64.jar
- - netty-resolver-dns-native-macos-4.1.116.Final-osx-x86_64.jar
+ - netty-resolver-dns-classes-macos-4.1.117.Final.jar
+ - netty-resolver-dns-native-macos-4.1.117.Final-osx-aarch_64.jar
+ - netty-resolver-dns-native-macos-4.1.117.Final-osx-x86_64.jar
* Prometheus client
- simpleclient-0.16.0.jar
- simpleclient_log4j2-0.16.0.jar
diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile
index 7019797037848..c3d9ab05ac466 100644
--- a/docker/pulsar/Dockerfile
+++ b/docker/pulsar/Dockerfile
@@ -97,8 +97,6 @@ RUN pip3 install pyyaml==6.0.1
# 4. /pulsar - hadoop writes to this directory
RUN mkdir /pulsar && chmod g+w /pulsar
-ENV PULSAR_ROOT_LOGGER=INFO,CONSOLE
-
COPY --from=pulsar /pulsar /pulsar
WORKDIR /pulsar
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 fe8cb238dbce4..139935054dab5 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
@@ -90,6 +90,8 @@ public class ManagedLedgerConfig {
@Getter
@Setter
private String shadowSourceName;
+ @Getter
+ private boolean persistIndividualAckAsLongArray;
public boolean isCreateIfMissing() {
return createIfMissing;
@@ -100,6 +102,11 @@ public ManagedLedgerConfig setCreateIfMissing(boolean createIfMissing) {
return this;
}
+ public ManagedLedgerConfig setPersistIndividualAckAsLongArray(boolean persistIndividualAckAsLongArray) {
+ this.persistIndividualAckAsLongArray = persistIndividualAckAsLongArray;
+ return this;
+ }
+
/**
* @return the lazyCursorRecovery
*/
@@ -505,8 +512,10 @@ public int getMaxUnackedRangesToPersistInMetadataStore() {
return maxUnackedRangesToPersistInMetadataStore;
}
- public void setMaxUnackedRangesToPersistInMetadataStore(int maxUnackedRangesToPersistInMetadataStore) {
+ public ManagedLedgerConfig setMaxUnackedRangesToPersistInMetadataStore(
+ int maxUnackedRangesToPersistInMetadataStore) {
this.maxUnackedRangesToPersistInMetadataStore = maxUnackedRangesToPersistInMetadataStore;
+ return this;
}
/**
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/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java
index 45219ca091fe0..3ddfc9bdcb57a 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,13 +35,14 @@
import java.time.Clock;
import java.util.ArrayDeque;
import java.util.ArrayList;
+import java.util.BitSet;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
@@ -59,7 +60,10 @@
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Function;
import java.util.function.Predicate;
+import java.util.stream.Collectors;
import java.util.stream.LongStream;
+import javax.annotation.Nullable;
+import lombok.Getter;
import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
import org.apache.bookkeeper.client.AsyncCallback.OpenCallback;
import org.apache.bookkeeper.client.BKException;
@@ -89,15 +93,17 @@
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound;
import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback;
import org.apache.bookkeeper.mledger.proto.MLDataFormats;
+import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongListMap;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo;
+import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo.Builder;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.StringProperty;
+import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.common.util.FutureUtil;
-import org.apache.pulsar.common.util.collections.BitSetRecyclable;
import org.apache.pulsar.common.util.collections.LongPairRangeSet;
import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer;
import org.apache.pulsar.common.util.collections.LongPairRangeSet.RangeBoundConsumer;
@@ -172,7 +178,8 @@ public class ManagedCursorImpl implements ManagedCursor {
protected volatile long messagesConsumedCounter;
// Current ledger used to append the mark-delete position
- private volatile LedgerHandle cursorLedger;
+ @VisibleForTesting
+ volatile LedgerHandle cursorLedger;
// Wether the current cursorLedger is read-only or writable
private boolean isCursorLedgerReadOnly = true;
@@ -199,7 +206,9 @@ public class ManagedCursorImpl implements ManagedCursor {
// Maintain the deletion status for batch messages
// (ledgerId, entryId) -> deletion indexes
- protected final ConcurrentSkipListMap batchDeletedIndexes;
+ @Getter
+ @VisibleForTesting
+ @Nullable protected final ConcurrentSkipListMap batchDeletedIndexes;
private final ReadWriteLock lock = new ReentrantReadWriteLock();
private RateLimiter markDeleteLimiter;
@@ -597,9 +606,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac
}
PositionImpl position = new PositionImpl(positionInfo);
- if (positionInfo.getIndividualDeletedMessagesCount() > 0) {
- recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList());
- }
+ recoverIndividualDeletedMessages(positionInfo);
if (getConfig().isDeletionAtBatchIndexLevelEnabled()
&& positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) {
recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList());
@@ -618,6 +625,60 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac
}
}
+ public void recoverIndividualDeletedMessages(PositionInfo positionInfo) {
+ if (positionInfo.getIndividualDeletedMessagesCount() > 0) {
+ recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList());
+ } else if (positionInfo.getIndividualDeletedMessageRangesCount() > 0) {
+ List rangeList = positionInfo.getIndividualDeletedMessageRangesList();
+ try {
+ Map rangeMap = rangeList.stream().collect(Collectors.toMap(LongListMap::getKey,
+ list -> list.getValuesList().stream().mapToLong(i -> i).toArray()));
+ // Guarantee compatability for the config "unackedRangesOpenCacheSetEnabled".
+ if (getConfig().isUnackedRangesOpenCacheSetEnabled()) {
+ individualDeletedMessages.build(rangeMap);
+ } else {
+ RangeSetWrapper rangeSetWrapperV2 = new RangeSetWrapper<>(positionRangeConverter,
+ positionRangeReverseConverter, true,
+ getConfig().isPersistentUnackedRangesWithMultipleEntriesEnabled());
+ rangeSetWrapperV2.build(rangeMap);
+ rangeSetWrapperV2.forEach(range -> {
+ individualDeletedMessages.addOpenClosed(range.lowerEndpoint().getLedgerId(),
+ range.lowerEndpoint().getEntryId(), range.upperEndpoint().getLedgerId(),
+ range.upperEndpoint().getEntryId());
+ return true;
+ });
+ rangeSetWrapperV2.clear();
+ }
+ } catch (Exception e) {
+ log.warn("[{}]-{} Failed to recover individualDeletedMessages from serialized data", ledger.getName(),
+ name, e);
+ }
+ }
+ }
+
+ private List buildLongPropertiesMap(Map properties) {
+ if (properties.isEmpty()) {
+ return Collections.emptyList();
+ }
+ List longListMap = new ArrayList<>();
+ MutableInt serializedSize = new MutableInt();
+ properties.forEach((id, ranges) -> {
+ if (ranges == null || ranges.length <= 0) {
+ return;
+ }
+ org.apache.bookkeeper.mledger.proto.MLDataFormats.LongListMap.Builder lmBuilder = LongListMap.newBuilder()
+ .setKey(id);
+ for (long range : ranges) {
+ lmBuilder.addValues(range);
+ }
+ LongListMap lm = lmBuilder.build();
+ longListMap.add(lm);
+ serializedSize.add(lm.getSerializedSize());
+ });
+ individualDeletedMessagesSerializedSize = serializedSize.toInteger();
+ return longListMap;
+ }
+
private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) {
lock.writeLock().lock();
try {
@@ -657,6 +718,7 @@ private void recoverIndividualDeletedMessages(List i
private void recoverBatchDeletedIndexes (
List batchDeletedIndexInfoList) {
+ Objects.requireNonNull(batchDeletedIndexes);
lock.writeLock().lock();
try {
this.batchDeletedIndexes.clear();
@@ -667,7 +729,7 @@ private void recoverBatchDeletedIndexes (
array[i] = batchDeletedIndexInfo.getDeleteSetList().get(i);
}
this.batchDeletedIndexes.put(PositionImpl.get(batchDeletedIndexInfo.getPosition().getLedgerId(),
- batchDeletedIndexInfo.getPosition().getEntryId()), BitSetRecyclable.create().resetWords(array));
+ batchDeletedIndexInfo.getPosition().getEntryId()), BitSet.valueOf(array));
}
});
} finally {
@@ -1327,13 +1389,11 @@ public void operationComplete() {
lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, isCompactionCursor()
? getProperties() : Collections.emptyMap(), null, null);
individualDeletedMessages.clear();
- if (getConfig().isDeletionAtBatchIndexLevelEnabled()) {
- batchDeletedIndexes.values().forEach(BitSetRecyclable::recycle);
+ if (batchDeletedIndexes != null) {
batchDeletedIndexes.clear();
long[] resetWords = newReadPosition.ackSet;
if (resetWords != null) {
- BitSetRecyclable ackSet = BitSetRecyclable.create().resetWords(resetWords);
- batchDeletedIndexes.put(newReadPosition, ackSet);
+ batchDeletedIndexes.put(newReadPosition, BitSet.valueOf(resetWords));
}
}
@@ -1964,41 +2024,7 @@ public void asyncMarkDelete(final Position position, Map propertie
log.debug("[{}] Mark delete cursor {} up to position: {}", ledger.getName(), name, position);
}
- PositionImpl newPosition = (PositionImpl) position;
-
- if (getConfig().isDeletionAtBatchIndexLevelEnabled()) {
- if (newPosition.ackSet != null) {
- AtomicReference bitSetRecyclable = new AtomicReference<>();
- BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(newPosition.ackSet);
- // In order to prevent the batch index recorded in batchDeletedIndexes from rolling back,
- // only update batchDeletedIndexes when the submitted batch index is greater
- // than the recorded index.
- batchDeletedIndexes.compute(newPosition,
- (k, v) -> {
- if (v == null) {
- return givenBitSet;
- }
- if (givenBitSet.nextSetBit(0) > v.nextSetBit(0)) {
- bitSetRecyclable.set(v);
- return givenBitSet;
- } else {
- bitSetRecyclable.set(givenBitSet);
- return v;
- }
- });
- if (bitSetRecyclable.get() != null) {
- bitSetRecyclable.get().recycle();
- }
- newPosition = ledger.getPreviousPosition(newPosition);
- }
- Map subMap = batchDeletedIndexes.subMap(PositionImpl.EARLIEST, newPosition);
- subMap.values().forEach(BitSetRecyclable::recycle);
- subMap.clear();
- } else if (newPosition.ackSet != null) {
- newPosition = ledger.getPreviousPosition(newPosition);
- newPosition.ackSet = null;
- }
-
+ PositionImpl newPosition = ackBatchPosition((PositionImpl) position);
if (((PositionImpl) ledger.getLastConfirmedEntry()).compareTo(newPosition) < 0) {
boolean shouldCursorMoveForward = false;
try {
@@ -2044,6 +2070,30 @@ public void asyncMarkDelete(final Position position, Map propertie
internalAsyncMarkDelete(newPosition, properties, callback, ctx);
}
+ private PositionImpl ackBatchPosition(PositionImpl position) {
+ return Optional.ofNullable(position.getAckSet())
+ .map(ackSet -> {
+ if (batchDeletedIndexes == null) {
+ return ledger.getPreviousPosition(position);
+ }
+ // In order to prevent the batch index recorded in batchDeletedIndexes from rolling back,
+ // only update batchDeletedIndexes when the submitted batch index is greater
+ // than the recorded index.
+ final var givenBitSet = BitSet.valueOf(ackSet);
+ batchDeletedIndexes.compute(position, (k, v) -> {
+ if (v == null || givenBitSet.nextSetBit(0) > v.nextSetBit(0)) {
+ return givenBitSet;
+ } else {
+ return v;
+ }
+ });
+ final var newPosition = ledger.getPreviousPosition(position);
+ batchDeletedIndexes.subMap(PositionImpl.EARLIEST, newPosition).clear();
+ return newPosition;
+ })
+ .orElse(position);
+ }
+
protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map properties,
final MarkDeleteCallback callback, final Object ctx) {
ledger.mbean.addMarkDeleteOp();
@@ -2149,12 +2199,10 @@ public void operationComplete() {
try {
individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(),
mdEntry.newPosition.getEntryId());
- if (getConfig().isDeletionAtBatchIndexLevelEnabled()) {
- Map subMap = batchDeletedIndexes.subMap(PositionImpl.EARLIEST,
+ if (batchDeletedIndexes != null) {
+ batchDeletedIndexes.subMap(PositionImpl.EARLIEST,
false, PositionImpl.get(mdEntry.newPosition.getLedgerId(),
- mdEntry.newPosition.getEntryId()), true);
- subMap.values().forEach(BitSetRecyclable::recycle);
- subMap.clear();
+ mdEntry.newPosition.getEntryId()), true).clear();
}
persistentMarkDeletePosition = mdEntry.newPosition;
} finally {
@@ -2289,11 +2337,8 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb
}
if (internalIsMessageDeleted(position)) {
- if (getConfig().isDeletionAtBatchIndexLevelEnabled()) {
- BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position);
- if (bitSetRecyclable != null) {
- bitSetRecyclable.recycle();
- }
+ if (batchDeletedIndexes != null) {
+ batchDeletedIndexes.remove(position);
}
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Position was already deleted {}", ledger.getName(), name, position);
@@ -2301,15 +2346,19 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb
continue;
}
if (position.ackSet == null) {
- if (getConfig().isDeletionAtBatchIndexLevelEnabled()) {
- BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position);
- if (bitSetRecyclable != null) {
- bitSetRecyclable.recycle();
- }
+ if (batchDeletedIndexes != null) {
+ batchDeletedIndexes.remove(position);
}
// Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will
// make the RangeSet recognize the "continuity" between adjacent Positions.
- PositionImpl previousPosition = ledger.getPreviousPosition(position);
+ // Before https://github.com/apache/pulsar/pull/21105 is merged, the range does not support crossing
+ // multi ledgers, so the first position's entryId maybe "-1".
+ PositionImpl previousPosition;
+ if (position.getEntryId() == 0) {
+ previousPosition = new PositionImpl(position.getLedgerId(), -1);
+ } else {
+ previousPosition = ledger.getPreviousPosition(position);
+ }
individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(),
previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId());
MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this);
@@ -2318,12 +2367,11 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb
log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name,
individualDeletedMessages);
}
- } else if (getConfig().isDeletionAtBatchIndexLevelEnabled()) {
- BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(position.ackSet);
- BitSetRecyclable bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> givenBitSet);
+ } else if (batchDeletedIndexes != null) {
+ final var givenBitSet = BitSet.valueOf(position.ackSet);
+ final var bitSet = batchDeletedIndexes.computeIfAbsent(position, __ -> givenBitSet);
if (givenBitSet != bitSet) {
bitSet.and(givenBitSet);
- givenBitSet.recycle();
}
if (bitSet.isEmpty()) {
PositionImpl previousPosition = ledger.getPreviousPosition(position);
@@ -2331,10 +2379,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb
previousPosition.getEntryId(),
position.getLedgerId(), position.getEntryId());
MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this);
- BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position);
- if (bitSetRecyclable != null) {
- bitSetRecyclable.recycle();
- }
+ batchDeletedIndexes.remove(position);
}
}
}
@@ -3079,7 +3124,7 @@ private List buildIndividualDeletedMessageRanges() {
private List buildBatchEntryDeletionIndexInfoList() {
lock.readLock().lock();
try {
- if (!getConfig().isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) {
+ if (batchDeletedIndexes == null || batchDeletedIndexes.isEmpty()) {
return Collections.emptyList();
}
MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo
@@ -3087,9 +3132,9 @@ private List buildBatchEntryDeletio
MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats
.BatchedEntryDeletionIndexInfo.newBuilder();
List result = new ArrayList<>();
- Iterator> iterator = batchDeletedIndexes.entrySet().iterator();
+ final var iterator = batchDeletedIndexes.entrySet().iterator();
while (iterator.hasNext() && result.size() < getConfig().getMaxBatchDeletedIndexToPersist()) {
- Map.Entry entry = iterator.next();
+ final var entry = iterator.next();
nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId());
nestedPositionBuilder.setEntryId(entry.getKey().getEntryId());
batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build());
@@ -3110,12 +3155,34 @@ private List buildBatchEntryDeletio
void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) {
PositionImpl position = mdEntry.newPosition;
- PositionInfo pi = PositionInfo.newBuilder().setLedgerId(position.getLedgerId())
+ Builder piBuilder = PositionInfo.newBuilder().setLedgerId(position.getLedgerId())
.setEntryId(position.getEntryId())
- .addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges())
.addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList())
- .addAllProperties(buildPropertiesMap(mdEntry.properties)).build();
-
+ .addAllProperties(buildPropertiesMap(mdEntry.properties));
+
+ Map internalRanges = null;
+ /**
+ * Cursor will create the {@link #individualDeletedMessages} typed {@link LongPairRangeSet.DefaultRangeSet} if
+ * disabled the config {@link ManagedLedgerConfig#unackedRangesOpenCacheSetEnabled}.
+ * {@link LongPairRangeSet.DefaultRangeSet} never implemented the methods below:
+ * - {@link LongPairRangeSet#toRanges(int)}, which is used to serialize cursor metadata.
+ * - {@link LongPairRangeSet#build(Map)}, which is used to deserialize cursor metadata.
+ * Do not enable the feature that https://github.com/apache/pulsar/pull/9292 introduced, to avoid serialization
+ * and deserialization error.
+ */
+ if (getConfig().isUnackedRangesOpenCacheSetEnabled() && getConfig().isPersistIndividualAckAsLongArray()) {
+ try {
+ internalRanges = individualDeletedMessages.toRanges(getConfig().getMaxUnackedRangesToPersist());
+ } catch (Exception e) {
+ log.warn("[{}]-{} Failed to serialize individualDeletedMessages", ledger.getName(), name, e);
+ }
+ }
+ if (internalRanges != null && !internalRanges.isEmpty()) {
+ piBuilder.addAllIndividualDeletedMessageRanges(buildLongPropertiesMap(internalRanges));
+ } else {
+ piBuilder.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges());
+ }
+ PositionInfo pi = piBuilder.build();
if (log.isDebugEnabled()) {
log.debug("[{}] Cursor {} Appending to ledger={} position={}", ledger.getName(), name, lh.getId(),
@@ -3487,11 +3554,11 @@ public long[] getBatchPositionAckSet(Position position) {
}
if (batchDeletedIndexes != null) {
- BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.get(position);
- if (bitSetRecyclable == null) {
+ final var bitSet = batchDeletedIndexes.get(position);
+ if (bitSet == null) {
return null;
} else {
- return bitSetRecyclable.toLongArray();
+ return bitSet.toLongArray();
}
} else {
return null;
@@ -3594,8 +3661,8 @@ private ManagedCursorImpl cursorImpl() {
@Override
public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) {
- if (getConfig().isDeletionAtBatchIndexLevelEnabled()) {
- BitSetRecyclable bitSet = batchDeletedIndexes.get(position);
+ if (batchDeletedIndexes != null) {
+ final var bitSet = batchDeletedIndexes.get(position);
return bitSet == null ? null : bitSet.toLongArray();
} else {
return null;
@@ -3720,9 +3787,9 @@ public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) thro
lock.readLock().unlock();
}
if (batchDeletedIndexes != null) {
- for (Map.Entry entry : this.batchDeletedIndexes.entrySet()) {
- BitSetRecyclable copiedBitSet = BitSetRecyclable.valueOf(entry.getValue());
- newNonDurableCursor.batchDeletedIndexes.put(entry.getKey(), copiedBitSet);
+ Objects.requireNonNull(newNonDurableCursor.batchDeletedIndexes);
+ for (final var entry : this.batchDeletedIndexes.entrySet()) {
+ newNonDurableCursor.batchDeletedIndexes.put(entry.getKey(), (BitSet) entry.getValue().clone());
}
}
return newNonDurableCursor;
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 74c53d7c1fd1a..7f062c8e682f4 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
@@ -199,7 +199,7 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore,
config.getCompressionConfigForManagedCursorInfo());
this.config = config;
this.mbean = new ManagedLedgerFactoryMBeanImpl(this);
- this.entryCacheManager = new RangeEntryCacheManagerImpl(this);
+ this.entryCacheManager = new RangeEntryCacheManagerImpl(this, scheduledExecutor);
this.statsTask = scheduledExecutor.scheduleWithFixedDelay(catchingAndLoggingThrowables(this::refreshStats),
0, config.getStatsPeriodSeconds(), TimeUnit.SECONDS);
this.flushCursorsTask = scheduledExecutor.scheduleAtFixedRate(catchingAndLoggingThrowables(this::flushCursors),
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java
index a55e6444b2fd9..76ac3e1be726c 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java
@@ -18,13 +18,12 @@
*/
package org.apache.bookkeeper.mledger.impl;
-import static java.util.Objects.requireNonNull;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Range;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
-import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
+import java.util.Map;
import org.apache.pulsar.common.util.collections.LongPairRangeSet;
import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet;
import org.roaringbitmap.RoaringBitSet;
@@ -39,7 +38,6 @@ public class RangeSetWrapper> implements LongPairRangeSe
private final LongPairRangeSet rangeSet;
private final LongPairConsumer rangeConverter;
- private final ManagedLedgerConfig config;
private final boolean enableMultiEntry;
/**
@@ -52,13 +50,19 @@ public class RangeSetWrapper> implements LongPairRangeSe
public RangeSetWrapper(LongPairConsumer rangeConverter,
RangeBoundConsumer rangeBoundConsumer,
ManagedCursorImpl managedCursor) {
- requireNonNull(managedCursor);
- this.config = managedCursor.getManagedLedger().getConfig();
+ this(rangeConverter, rangeBoundConsumer, managedCursor.getConfig().isUnackedRangesOpenCacheSetEnabled(),
+ managedCursor.getConfig().isPersistentUnackedRangesWithMultipleEntriesEnabled());
+ }
+
+ public RangeSetWrapper(LongPairConsumer rangeConverter,
+ RangeBoundConsumer rangeBoundConsumer,
+ boolean unackedRangesOpenCacheSetEnabled,
+ boolean persistentUnackedRangesWithMultipleEntriesEnabled) {
this.rangeConverter = rangeConverter;
- this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled()
+ this.rangeSet = unackedRangesOpenCacheSetEnabled
? new OpenLongPairRangeSet<>(rangeConverter, RoaringBitSet::new)
: new LongPairRangeSet.DefaultRangeSet<>(rangeConverter, rangeBoundConsumer);
- this.enableMultiEntry = config.isPersistentUnackedRangesWithMultipleEntriesEnabled();
+ this.enableMultiEntry = persistentUnackedRangesWithMultipleEntriesEnabled;
}
@Override
@@ -142,6 +146,16 @@ public Range lastRange() {
return rangeSet.lastRange();
}
+ @Override
+ public Map toRanges(int maxRanges) {
+ return rangeSet.toRanges(maxRanges);
+ }
+
+ @Override
+ public void build(Map internalRange) {
+ rangeSet.build(internalRange);
+ }
+
@Override
public int cardinality(long lowerKey, long lowerValue, long upperKey, long upperValue) {
return rangeSet.cardinality(lowerKey, lowerValue, upperKey, upperValue);
@@ -176,4 +190,22 @@ public boolean isDirtyLedgers(long ledgerId) {
public String toString() {
return rangeSet.toString();
}
+
+ @Override
+ public int hashCode() {
+ return rangeSet.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof RangeSetWrapper)) {
+ return false;
+ }
+ if (this == obj) {
+ return true;
+ }
+ @SuppressWarnings("rawtypes")
+ RangeSetWrapper set = (RangeSetWrapper) obj;
+ return this.rangeSet.equals(set.rangeSet);
+ }
}
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 b946dc09a0c71..3a6bb3cd039c3 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
@@ -20,9 +20,13 @@
import com.google.common.annotations.VisibleForTesting;
import io.prometheus.client.Gauge;
-import lombok.AllArgsConstructor;
-import lombok.ToString;
+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.extern.slf4j.Slf4j;
+import org.jctools.queues.SpscArrayQueue;
@Slf4j
public class InflightReadsLimiter {
@@ -41,15 +45,35 @@ public class InflightReadsLimiter {
private final long maxReadsInFlightSize;
private long remainingBytes;
+ private final long acquireTimeoutMillis;
+ private final ScheduledExecutorService timeOutExecutor;
+ private final boolean enabled;
- public InflightReadsLimiter(long maxReadsInFlightSize) {
- if (maxReadsInFlightSize <= 0) {
+ record Handle(long permits, long creationTime, 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) {
+ 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;
}
@VisibleForTesting
@@ -57,70 +81,178 @@ public synchronized long getRemainingBytes() {
return remainingBytes;
}
- @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 (permits > maxReadsInFlightSize && remainingBytes == maxReadsInFlightSize) {
+ remainingBytes = 0;
+ if (log.isInfoEnabled()) {
+ log.info("Requested permits {} exceeded maxReadsInFlightSize {}, creationTime: {}, remainingBytes:{}. "
+ + "Allowing request with permits set to maxReadsInFlightSize.",
+ permits, maxReadsInFlightSize, handle.creationTime, remainingBytes);
+ }
+ updateMetrics();
+ return Optional.of(new Handle(maxReadsInFlightSize, handle.creationTime, true));
+ } 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;
+ }
+ if (!timeoutCheckRunning) {
+ timeoutCheckRunning = true;
+ timeOutExecutor.schedule(this::timeoutCheck, delayMillis, TimeUnit.MILLISECONDS);
}
- 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());
- }
+ }
+
+ 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);
+ }
+ try {
+ queuedHandle.callback.accept(new Handle(0, queuedHandle.handle.creationTime, false));
+ } catch (Exception e) {
+ log.error("Error in callback of timed out queued permits: {}, creationTime: {}, remainingBytes:{}",
+ queuedHandle.handle.permits, queuedHandle.handle.creationTime, remainingBytes, e);
+ }
}
- 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) {
+ boolean timedOut = acquireTimeoutMillis > 0
+ && System.currentTimeMillis() - queuedHandle.handle.creationTime > acquireTimeoutMillis;
+ if (timedOut) {
+ // remove the peeked handle from the queue
+ queuedHandles.poll();
+ handleTimeout(queuedHandle);
+ } else if (remainingBytes >= queuedHandle.handle.permits
+ || queuedHandle.handle.permits > maxReadsInFlightSize
+ && remainingBytes == maxReadsInFlightSize) {
+ // remove the peeked handle from the queue
+ queuedHandles.poll();
+ handleQueuedHandle(queuedHandle);
+ } else {
+ break;
+ }
+ } else {
+ break;
+ }
+ }
+ updateMetrics();
+ }
+
+ private void handleQueuedHandle(QueuedHandle queuedHandle) {
+ long permits = queuedHandle.handle.permits;
+ Handle handleForCallback = queuedHandle.handle;
+ if (permits > maxReadsInFlightSize && remainingBytes == maxReadsInFlightSize) {
+ remainingBytes = 0;
+ if (log.isInfoEnabled()) {
+ log.info("Requested permits {} exceeded maxReadsInFlightSize {}, creationTime: {}, remainingBytes:{}. "
+ + "Allowing request with permits set to maxReadsInFlightSize.",
+ permits, maxReadsInFlightSize, queuedHandle.handle.creationTime, remainingBytes);
+ }
+ handleForCallback = new Handle(maxReadsInFlightSize, queuedHandle.handle.creationTime, true);
+ } else {
+ remainingBytes -= permits;
+ if (log.isDebugEnabled()) {
+ log.debug("acquired queued permits: {}, creationTime: {}, remainingBytes:{}",
+ permits, queuedHandle.handle.creationTime, remainingBytes);
+ }
+ }
+ try {
+ queuedHandle.callback.accept(handleForCallback);
+ } catch (Exception e) {
+ log.error("Error in callback of acquired queued permits: {}, creationTime: {}, remainingBytes:{}",
+ handleForCallback.permits, handleForCallback.creationTime, remainingBytes, e);
}
}
@@ -130,8 +262,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..5944199287ee1 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
@@ -25,9 +25,8 @@
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicBoolean;
-import lombok.AllArgsConstructor;
-import lombok.Value;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
@@ -95,15 +94,11 @@ public PendingReadsManager(RangeEntryCacheImpl rangeEntryCache) {
this.rangeEntryCache = rangeEntryCache;
}
- @Value
- private static class PendingReadKey {
- private final long startEntry;
- private final long endEntry;
+ private record PendingReadKey(long startEntry, long endEntry) {
long size() {
return endEntry - startEntry + 1;
}
-
boolean includes(PendingReadKey other) {
return startEntry <= other.startEntry && other.endEntry <= endEntry;
}
@@ -135,25 +130,18 @@ PendingReadKey reminderOnRight(PendingReadKey other) {
}
- @AllArgsConstructor
- private static final class ReadEntriesCallbackWithContext {
- final AsyncCallbacks.ReadEntriesCallback callback;
- final Object ctx;
- final long startEntry;
- final long endEntry;
+ private record ReadEntriesCallbackWithContext(AsyncCallbacks.ReadEntriesCallback callback, Object ctx,
+ long startEntry, long endEntry) {
}
- @AllArgsConstructor
- private static final class FindPendingReadOutcome {
- final PendingRead pendingRead;
- final PendingReadKey missingOnLeft;
- final PendingReadKey missingOnRight;
+ private record FindPendingReadOutcome(PendingRead pendingRead,
+ PendingReadKey missingOnLeft, PendingReadKey missingOnRight) {
boolean needsAdditionalReads() {
return missingOnLeft != null || missingOnRight != null;
}
}
- private FindPendingReadOutcome findPendingRead(PendingReadKey key, Map ledgerCache, AtomicBoolean created) {
synchronized (ledgerCache) {
PendingRead existing = ledgerCache.get(key);
@@ -222,18 +210,74 @@ private FindPendingReadOutcome findPendingRead(PendingReadKey key, Map ledgerCache;
+ final ConcurrentMap ledgerCache;
final List callbacks = new ArrayList<>(1);
boolean completed = false;
public PendingRead(PendingReadKey key,
- Map ledgerCache) {
+ ConcurrentMap ledgerCache) {
this.key = key;
this.ledgerCache = ledgerCache;
}
- private List keepEntries(List list, long startEntry, long endEntry) {
- List result = new ArrayList<>((int) (endEntry - startEntry));
+ public void attach(CompletableFuture> handle) {
+ handle.whenComplete((entriesToReturn, error) -> {
+ // execute in the completing thread
+ completeAndRemoveFromCache();
+ // execute the callbacks in the managed ledger executor
+ rangeEntryCache.getManagedLedger().getExecutor().execute(() -> {
+ if (error != null) {
+ readEntriesFailed(error);
+ } else {
+ readEntriesComplete(entriesToReturn);
+ }
+ });
+ });
+ }
+
+ private synchronized void completeAndRemoveFromCache() {
+ completed = true;
+ // When the read has completed, remove the instance from the ledgerCache map
+ // so that new reads will go to a new instance.
+ // this is required because we are going to do refcount management
+ // on the results of the callback
+ ledgerCache.remove(key, this);
+ }
+
+ private synchronized void readEntriesComplete(List entriesToReturn) {
+ if (callbacks.size() == 1) {
+ ReadEntriesCallbackWithContext first = callbacks.get(0);
+ 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);
+ } else {
+ first.callback.readEntriesComplete(
+ keepEntries(entriesToReturn, first.startEntry, first.endEntry),
+ first.ctx);
+ }
+ } else {
+ for (ReadEntriesCallbackWithContext callback : callbacks) {
+ callback.callback.readEntriesComplete(
+ copyEntries(entriesToReturn, callback.startEntry, callback.endEntry),
+ callback.ctx);
+ }
+ for (EntryImpl entry : entriesToReturn) {
+ entry.release();
+ }
+ }
+ }
+
+ private synchronized void readEntriesFailed(Throwable error) {
+ for (ReadEntriesCallbackWithContext callback : callbacks) {
+ ManagedLedgerException mlException = createManagedLedgerException(error);
+ callback.callback.readEntriesFailed(mlException, callback.ctx);
+ }
+ }
+
+ private List keepEntries(List list, long startEntry, long endEntry) {
+ List result = new ArrayList<>((int) (endEntry - startEntry));
for (EntryImpl entry : list) {
long entryId = entry.getEntryId();
if (startEntry <= entryId && entryId <= endEntry) {
@@ -245,62 +289,16 @@ private List keepEntries(List list, long startEntry, long
return result;
}
- 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
- // on the results of the callback
- handle.whenComplete((___, error) -> {
- synchronized (PendingRead.this) {
- completed = true;
- synchronized (ledgerCache) {
- ledgerCache.remove(key, this);
- }
- }
- });
-
- handle.thenAcceptAsync(entriesToReturn -> {
- synchronized (PendingRead.this) {
- if (callbacks.size() == 1) {
- ReadEntriesCallbackWithContext first = callbacks.get(0);
- 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);
- } else {
- first.callback.readEntriesComplete(
- (List) keepEntries(entriesToReturn, first.startEntry, first.endEntry),
- 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) {
- long entryId = entry.getEntryId();
- if (callbackStartEntry <= entryId && entryId <= callbackEndEntry) {
- EntryImpl entryCopy = EntryImpl.create(entry);
- copy.add(entryCopy);
- }
- }
- callback.callback.readEntriesComplete((List) copy, callback.ctx);
- }
- for (EntryImpl entry : entriesToReturn) {
- entry.release();
- }
- }
- }
- }, rangeEntryCache.getManagedLedger().getExecutor()).exceptionally(exception -> {
- synchronized (PendingRead.this) {
- for (ReadEntriesCallbackWithContext callback : callbacks) {
- ManagedLedgerException mlException = createManagedLedgerException(exception);
- callback.callback.readEntriesFailed(mlException, callback.ctx);
- }
+ private List copyEntries(List entriesToReturn, long startEntry, long endEntry) {
+ List result = new ArrayList<>((int) (endEntry - startEntry + 1));
+ for (EntryImpl entry : entriesToReturn) {
+ long entryId = entry.getEntryId();
+ if (startEntry <= entryId && entryId <= endEntry) {
+ EntryImpl entryCopy = EntryImpl.create(entry);
+ result.add(entryCopy);
}
- return null;
- });
+ }
+ return result;
}
synchronized boolean addListener(AsyncCallbacks.ReadEntriesCallback callback,
@@ -318,7 +316,7 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldC
final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) {
final PendingReadKey key = new PendingReadKey(firstEntry, lastEntry);
- Map pendingReadsForLedger =
+ ConcurrentMap pendingReadsForLedger =
cachedPendingReads.computeIfAbsent(lh.getId(), (l) -> new ConcurrentHashMap<>());
boolean listenerAdded = false;
@@ -362,7 +360,7 @@ public void readEntriesFailed(ManagedLedgerException exception,
};
rangeEntryCache.asyncReadEntry0(lh,
missingOnRight.startEntry, missingOnRight.endEntry,
- shouldCacheEntry, readFromRightCallback, null);
+ shouldCacheEntry, readFromRightCallback, null, false);
}
@Override
@@ -372,7 +370,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object dummyCtx4
}
};
rangeEntryCache.asyncReadEntry0(lh, missingOnLeft.startEntry, missingOnLeft.endEntry,
- shouldCacheEntry, readFromLeftCallback, null);
+ shouldCacheEntry, readFromLeftCallback, null, false);
} else if (missingOnLeft != null) {
AsyncCallbacks.ReadEntriesCallback readFromLeftCallback =
new AsyncCallbacks.ReadEntriesCallback() {
@@ -395,7 +393,7 @@ public void readEntriesFailed(ManagedLedgerException exception,
}
};
rangeEntryCache.asyncReadEntry0(lh, missingOnLeft.startEntry, missingOnLeft.endEntry,
- shouldCacheEntry, readFromLeftCallback, null);
+ shouldCacheEntry, readFromLeftCallback, null, false);
} else if (missingOnRight != null) {
AsyncCallbacks.ReadEntriesCallback readFromRightCallback =
new AsyncCallbacks.ReadEntriesCallback() {
@@ -418,7 +416,7 @@ public void readEntriesFailed(ManagedLedgerException exception,
}
};
rangeEntryCache.asyncReadEntry0(lh, missingOnRight.startEntry, missingOnRight.endEntry,
- shouldCacheEntry, readFromRightCallback, null);
+ shouldCacheEntry, readFromRightCallback, null, false);
}
}
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 21eb62e5a8caa..c8d14cebebc88 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,18 +22,19 @@
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 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.Entry;
@@ -56,7 +57,9 @@ public class RangeEntryCacheImpl implements EntryCache {
/**
* Overhead per-entry to take into account the envelope.
*/
- private static final long BOOKKEEPER_READ_OVERHEAD_PER_ENTRY = 64;
+ public 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;
@@ -65,18 +68,16 @@ public class RangeEntryCacheImpl implements EntryCache {
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;
+ private final LongAdder totalAddedEntriesSize = new LongAdder();
+ private final LongAdder totalAddedEntriesCount = new LongAdder();
+
public RangeEntryCacheImpl(RangeEntryCacheManagerImpl manager, ManagedLedgerImpl ml, boolean copyEntries) {
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.copyEntries = copyEntries;
@@ -101,7 +102,7 @@ public String getName() {
}
@VisibleForTesting
- InflightReadsLimiter getPendingReadsLimiter() {
+ public InflightReadsLimiter getPendingReadsLimiter() {
return manager.getInflightReadsLimiter();
}
@@ -117,17 +118,18 @@ InflightReadsLimiter getPendingReadsLimiter() {
@Override
public boolean insert(EntryImpl entry) {
+ int entryLength = entry.getLength();
if (!manager.hasSpaceInCache()) {
if (log.isDebugEnabled()) {
log.debug("[{}] Skipping cache while doing eviction: {} - size: {}", ml.getName(), entry.getPosition(),
- entry.getLength());
+ entryLength);
}
return false;
}
if (log.isDebugEnabled()) {
log.debug("[{}] Adding entry to cache: {} - size: {}", ml.getName(), entry.getPosition(),
- entry.getLength());
+ entryLength);
}
PositionImpl position = entry.getPosition();
@@ -149,7 +151,9 @@ public boolean insert(EntryImpl entry) {
EntryImpl cacheEntry = EntryImpl.create(position, cachedData);
cachedData.release();
if (entries.put(position, cacheEntry)) {
- manager.entryAdded(entry.getLength());
+ totalAddedEntriesSize.add(entryLength);
+ totalAddedEntriesCount.increment();
+ manager.entryAdded(entryLength);
return true;
} else {
// entry was not inserted into cache, we need to discard it
@@ -225,7 +229,23 @@ public void invalidateAllEntries(long ledgerId) {
public void asyncReadEntry(ReadHandle lh, PositionImpl position, final ReadEntryCallback callback,
final Object ctx) {
try {
- asyncReadEntry0(lh, position, callback, ctx);
+ asyncReadEntriesByPosition(lh, position, position, 1,
+ DEFAULT_CACHE_INDIVIDUAL_READ_ENTRY,
+ 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
@@ -236,52 +256,11 @@ public void asyncReadEntry(ReadHandle lh, PositionImpl position, final ReadEntry
}
}
- private void asyncReadEntry0(ReadHandle lh, PositionImpl 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,
final ReadEntriesCallback callback, Object ctx) {
try {
- asyncReadEntry0(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx);
+ asyncReadEntry0(lh, firstEntry, lastEntry, shouldCacheEntry, 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,34 +273,123 @@ 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);
+ final ReadEntriesCallback callback, Object ctx, boolean acquirePermits) {
+ final long ledgerId = lh.getId();
+ final int numberOfEntries = (int) (lastEntry - firstEntry) + 1;
+ final PositionImpl firstPosition = PositionImpl.get(ledgerId, firstEntry);
+ final PositionImpl lastPosition = PositionImpl.get(ledgerId, lastEntry);
+ asyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, shouldCacheEntry, 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, PositionImpl firstPosition, PositionImpl lastPosition,
+ int numberOfEntries, boolean shouldCacheEntry,
+ 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 PositionImpl firstPosition = PositionImpl.get(lh.getId(), firstEntry);
- final PositionImpl lastPosition = PositionImpl.get(lh.getId(), lastEntry);
+ InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter();
+ if (!acquirePermits || pendingReadsLimiter.isDisabled()) {
+ doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, shouldCacheEntry,
+ 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,
+ shouldCacheEntry, originalCallback, ctx, handle, estimatedReadSize);
+ });
+ });
+ // permits were immediately available and acquired
+ if (optionalHandle.isPresent()) {
+ doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries,
+ shouldCacheEntry, originalCallback, ctx, optionalHandle.get(), estimatedReadSize);
+ }
+ }
+ }
- if (log.isDebugEnabled()) {
- log.debug("[{}] Reading entries range ledger {}: {} to {}", ml.getName(), ledgerId, firstEntry, lastEntry);
+ void doAsyncReadEntriesWithAcquiredPermits(ReadHandle lh, PositionImpl firstPosition, PositionImpl lastPosition,
+ int numberOfEntries, boolean shouldCacheEntry,
+ final ReadEntriesCallback originalCallback, Object ctx,
+ InflightReadsLimiter.Handle handle, long estimatedReadSize) {
+ if (!handle.success()) {
+ String message = String.format(
+ "Couldn't acquire enough permits on the max reads in flight limiter to read from ledger "
+ + "%d, %s, estimated read size %d bytes for %d entries (check "
+ + "managedLedgerMaxReadsInFlightSizeInMB, "
+ + "managedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis and "
+ + "managedLedgerMaxReadsInFlightPermitsAcquireQueueSize)", lh.getId(), getName(),
+ estimatedReadSize, numberOfEntries);
+ 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, shouldCacheEntry,
+ wrappedCallback, ctx);
+ }
+
+ void doAsyncReadEntriesByPosition(ReadHandle lh, PositionImpl firstPosition, PositionImpl lastPosition,
+ int numberOfEntries, boolean shouldCacheEntry, final ReadEntriesCallback callback,
+ Object ctx) {
+ Collection cachedEntries;
+ if (firstPosition.compareTo(lastPosition) == 0) {
+ EntryImpl cachedEntry = entries.get(firstPosition);
+ if (cachedEntry == null) {
+ cachedEntries = Collections.emptyList();
+ } else {
+ cachedEntries = Collections.singleton(cachedEntry);
+ }
+ } else {
+ cachedEntries = entries.getRange(firstPosition, lastPosition);
+ }
- if (cachedEntries.size() == entriesToRead) {
+ 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) {
@@ -332,11 +400,11 @@ 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 {
if (!cachedEntries.isEmpty()) {
@@ -344,77 +412,24 @@ void asyncReadEntry0WithLimits(ReadHandle lh, long firstEntry, long lastEntry, b
}
// Read all the entries from bookkeeper
- pendingReadsManager.readEntries(lh, firstEntry, lastEntry,
+ pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(),
shouldCacheEntry, 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;
+ @VisibleForTesting
+ public 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;
}
/**
@@ -437,8 +452,7 @@ 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);
for (LedgerEntry e : ledgerEntries) {
EntryImpl entry = RangeEntryCacheManagerImpl.create(e, interceptor);
entriesToReturn.add(entry);
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 d5a3019855cb5..4fff47df82204 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
@@ -27,7 +27,9 @@
import java.util.concurrent.atomic.AtomicLong;
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;
@@ -56,12 +58,15 @@ public class RangeEntryCacheManagerImpl implements EntryCacheManager {
private static final double evictionTriggerThresholdPercent = 0.98;
- public RangeEntryCacheManagerImpl(ManagedLedgerFactoryImpl factory) {
- this.maxSize = factory.getConfig().getMaxCacheSize();
- this.inflightReadsLimiter = new InflightReadsLimiter(
- factory.getConfig().getManagedLedgerMaxReadsInFlightSize());
+ public RangeEntryCacheManagerImpl(ManagedLedgerFactoryImpl factory, OrderedScheduler scheduledExecutor) {
+ ManagedLedgerFactoryConfig config = factory.getConfig();
+ this.maxSize = config.getMaxCacheSize();
+ this.inflightReadsLimiter = new InflightReadsLimiter(config.getManagedLedgerMaxReadsInFlightSize(),
+ config.getManagedLedgerMaxReadsInFlightPermitsAcquireQueueSize(),
+ config.getManagedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis(),
+ scheduledExecutor);
this.evictionTriggerThreshold = (long) (maxSize * evictionTriggerThresholdPercent);
- this.cacheEvictionWatermark = factory.getConfig().getCacheEvictionWatermark();
+ this.cacheEvictionWatermark = config.getCacheEvictionWatermark();
this.evictionPolicy = new EntryCacheDefaultEvictionPolicy();
this.mlFactory = factory;
this.mlFactoryMBean = factory.getMbean();
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
index 2f2b161a30684..0de6f94362215 100644
--- 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
@@ -106,7 +106,39 @@ K getKey() {
return localKey;
}
+ /**
+ * Get the value associated with the key. Returns null if the key does not match the key.
+ *
+ * @param key the key to match
+ * @return the value associated with the key, or null if the value has already been recycled or the key does not
+ * match
+ */
V getValue(K key) {
+ return getValueInternal(key, false);
+ }
+
+ /**
+ * Get the value associated with the Map.Entry's key and value. Exact instance of the key is required to match.
+ * @param entry the entry which contains the key and {@link EntryWrapper} value to get the value from
+ * @return the value associated with the key, or null if the value has already been recycled or the key does not
+ * exactly match the same instance
+ */
+ static V getValueMatchingMapEntry(Map.Entry> entry) {
+ return entry.getValue().getValueInternal(entry.getKey(), true);
+ }
+
+ /**
+ * Get the value associated with the key. Returns null if the key does not match the key associated with the
+ * value.
+ *
+ * @param key the key to match
+ * @param requireSameKeyInstance when true, the matching will be restricted to exactly the same instance of the
+ * key as the one stored in the wrapper. This is used to avoid any races
+ * when retrieving or removing the entries from the cache when the key and value
+ * instances are available.
+ * @return the value associated with the key, or null if the key does not match
+ */
+ private V getValueInternal(K key, boolean requireSameKeyInstance) {
long stamp = lock.tryOptimisticRead();
K localKey = this.key;
V localValue = this.value;
@@ -116,7 +148,11 @@ V getValue(K key) {
localValue = this.value;
lock.unlockRead(stamp);
}
- if (localKey != key) {
+ // check that the given key matches the key associated with the value in the entry
+ // this is used to detect if the entry has already been recycled and contains another key
+ // when requireSameKeyInstance is true, the key must be exactly the same instance as the one stored in the
+ // entry to match
+ if (localKey != key && (requireSameKeyInstance || localKey == null || !localKey.equals(key))) {
return null;
}
return localValue;
@@ -236,34 +272,45 @@ public boolean exists(Key key) {
* The caller is responsible for releasing the reference.
*/
public Value get(Key key) {
- return getValue(key, entries.get(key));
+ return getValueFromWrapper(key, entries.get(key));
}
- private Value getValue(Key key, EntryWrapper valueWrapper) {
+ private Value getValueFromWrapper(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;
- }
+ return getRetainedValueMatchingKey(key, value);
+ }
+ }
+
+ private Value getValueMatchingEntry(Map.Entry> entry) {
+ Value valueMatchingEntry = EntryWrapper.getValueMatchingMapEntry(entry);
+ return getRetainedValueMatchingKey(entry.getKey(), valueMatchingEntry);
+ }
+
+ // validates that the value matches the key and that the value has not been recycled
+ // which are possible due to the lack of exclusive locks in the cache and the use of reference counted objects
+ private Value getRetainedValueMatchingKey(Key key, Value value) {
+ if (value == null) {
+ // the wrapper has been recycled and contains another key
+ return null;
+ }
+ try {
+ value.retain();
+ } catch (IllegalReferenceCountException e) {
+ // Value was already deallocated
+ return null;
+ }
+ // check that the value matches the key and that there's at least 2 references to it since
+ // the cache should be holding one reference and a new reference was just added in this method
+ if (value.refCnt() > 1 && value.matchesKey(key)) {
+ return value;
+ } else {
+ // Value or IdentityWrapper was recycled and already contains another value
+ // release the reference added in this method
+ value.release();
+ return null;
}
}
@@ -280,7 +327,7 @@ public Collection getRange(Key first, Key last) {
// 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());
+ Value value = getValueMatchingEntry(entry);
if (value != null) {
values.add(value);
}
@@ -297,6 +344,9 @@ public Collection getRange(Key first, Key last) {
* @return an pair of ints, containing the number of removed entries and the total size
*/
public Pair removeRange(Key first, Key last, boolean lastInclusive) {
+ if (log.isDebugEnabled()) {
+ log.debug("Removing entries in range [{}, {}], lastInclusive: {}", first, last, lastInclusive);
+ }
RemovalCounters counters = RemovalCounters.create();
Map> subMap = entries.subMap(first, true, last, lastInclusive);
for (Map.Entry> entry : subMap.entrySet()) {
@@ -320,7 +370,7 @@ private RemoveEntryResult removeEntry(Map.Entry> e
boolean skipInvalid, Predicate removeCondition) {
Key key = entry.getKey();
EntryWrapper entryWrapper = entry.getValue();
- Value value = entryWrapper.getValue(key);
+ Value value = getValueMatchingEntry(entry);
if (value == null) {
// the wrapper has already been recycled and contains another key
if (!skipInvalid) {
@@ -404,6 +454,9 @@ private Pair handleRemovalResult(RemovalCounters counters) {
* @return a pair containing the number of entries evicted and their total size
*/
public Pair evictLeastAccessedEntries(long minSize) {
+ if (log.isDebugEnabled()) {
+ log.debug("Evicting entries to reach a minimum size of {}", minSize);
+ }
checkArgument(minSize > 0);
RemovalCounters counters = RemovalCounters.create();
while (counters.removedSize < minSize && !Thread.currentThread().isInterrupted()) {
@@ -422,6 +475,9 @@ public Pair evictLeastAccessedEntries(long minSize) {
* @return the tota
*/
public Pair evictLEntriesBeforeTimestamp(long maxTimestamp) {
+ if (log.isDebugEnabled()) {
+ log.debug("Evicting entries with timestamp <= {}", maxTimestamp);
+ }
RemovalCounters counters = RemovalCounters.create();
while (!Thread.currentThread().isInterrupted()) {
Map.Entry> entry = entries.firstEntry();
@@ -453,6 +509,9 @@ public long getSize() {
* @return size of removed entries
*/
public Pair clear() {
+ if (log.isDebugEnabled()) {
+ log.debug("Clearing the cache with {} entries and size {}", entries.size(), size.get());
+ }
RemovalCounters counters = RemovalCounters.create();
while (!Thread.currentThread().isInterrupted()) {
Map.Entry> entry = entries.firstEntry();
diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto
index c4e502819fa9e..c05a6441bd8c7 100644
--- a/managed-ledger/src/main/proto/MLDataFormats.proto
+++ b/managed-ledger/src/main/proto/MLDataFormats.proto
@@ -82,6 +82,7 @@ message PositionInfo {
// Store which index in the batch message has been deleted
repeated BatchedEntryDeletionIndexInfo batchedEntryDeletionIndexInfo = 5;
+ repeated LongListMap individualDeletedMessageRanges = 6;
}
message NestedPositionInfo {
@@ -89,6 +90,11 @@ message NestedPositionInfo {
required int64 entryId = 2;
}
+message LongListMap {
+ required int64 key = 1;
+ repeated int64 values = 2;
+}
+
message MessageRange {
required NestedPositionInfo lowerEndpoint = 1;
required NestedPositionInfo upperEndpoint = 2;
diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java
new file mode 100644
index 0000000000000..48f0cf08ddff4
--- /dev/null
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.bookkeeper.mledger.impl;
+
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.doAnswer;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig;
+import org.apache.bookkeeper.mledger.impl.cache.InflightReadsLimiter;
+import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl;
+import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl;
+import org.apache.bookkeeper.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.mockito.Mockito;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class InflightReadsLimiterIntegrationTest extends MockedBookKeeperTestCase {
+
+ @DataProvider
+ public Object[][] readMissingCases() {
+ return new Object[][]{
+ {"missRight"},
+ {"missLeft"},
+ {"bothMiss"}
+ };
+ }
+
+ @Test(dataProvider = "readMissingCases")
+ public void testPreciseLimitation(String missingCase) throws Exception {
+ final long start1 = 50;
+ final long start2 = "missLeft".endsWith(missingCase) || "bothMiss".equals(missingCase) ? 30 : 50;
+ final long end1 = 99;
+ final long end2 = "missRight".endsWith(missingCase) || "bothMiss".equals(missingCase) ? 109 : 99;
+ final HashSet secondReadEntries = new HashSet<>();
+ if (start2 < start1) {
+ secondReadEntries.add(start2);
+ }
+ if (end2 > end1) {
+ secondReadEntries.add(end1 + 1);
+ }
+ final int readCount1 = (int) (end1 - start1 + 1);
+ final int readCount2 = (int) (end2 - start2 + 1);
+
+ final DefaultThreadFactory threadFactory = new DefaultThreadFactory(UUID.randomUUID().toString());
+ final ManagedLedgerConfig config = new ManagedLedgerConfig();
+ config.setMaxEntriesPerLedger(100000);
+ ManagedLedgerFactoryConfig factoryConfig = new ManagedLedgerFactoryConfig();
+ factoryConfig.setCacheEvictionIntervalMs(3600 * 1000);
+ factoryConfig.setManagedLedgerMaxReadsInFlightSize(1000_000);
+ final ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConfig);
+ final ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open("my_test_ledger", config);
+ final RangeEntryCacheImpl entryCache = (RangeEntryCacheImpl) ml.entryCache;
+ final RangeEntryCacheManagerImpl rangeEntryCacheManager =
+ (RangeEntryCacheManagerImpl) factory.getEntryCacheManager();
+ final InflightReadsLimiter limiter = rangeEntryCacheManager.getInflightReadsLimiter();
+ final long totalCapacity =limiter.getRemainingBytes();
+ // final ManagedCursorImpl c1 = (ManagedCursorImpl) ml.openCursor("c1");
+ for (byte i = 1; i < 127; i++) {
+ log.info("add entry: " + i);
+ ml.addEntry(new byte[]{i});
+ }
+ // Evict cached entries.
+ entryCache.evictEntries(ml.currentLedgerSize);
+ Assert.assertEquals(entryCache.getSize(), 0);
+
+ CountDownLatch readCompleteSignal1 = new CountDownLatch(1);
+ CountDownLatch readCompleteSignal2 = new CountDownLatch(1);
+ CountDownLatch firstReadingStarted = new CountDownLatch(1);
+ LedgerHandle currentLedger = ml.currentLedger;
+ LedgerHandle spyCurrentLedger = Mockito.spy(currentLedger);
+ ml.currentLedger = spyCurrentLedger;
+ Answer answer = invocation -> {
+ long firstEntry = (long) invocation.getArguments()[0];
+ log.info("reading entry: {}", firstEntry);
+ if (firstEntry == start1) {
+ // Wait 3s to make
+ firstReadingStarted.countDown();
+ readCompleteSignal1.await();
+ Object res = invocation.callRealMethod();
+ return res;
+ } else if(secondReadEntries.contains(firstEntry)) {
+ final CompletableFuture res = new CompletableFuture<>();
+ threadFactory.newThread(() -> {
+ try {
+ readCompleteSignal2.await();
+ CompletableFuture future =
+ (CompletableFuture) invocation.callRealMethod();
+ future.thenAccept(v -> {
+ res.complete(v);
+ }).exceptionally(ex -> {
+ res.completeExceptionally(ex);
+ return null;
+ });
+ } catch (Throwable ex) {
+ res.completeExceptionally(ex);
+ }
+ }).start();
+ return res;
+ } else {
+ return invocation.callRealMethod();
+ }
+ };
+ doAnswer(answer).when(spyCurrentLedger).readAsync(anyLong(), anyLong());
+ doAnswer(answer).when(spyCurrentLedger).readUnconfirmedAsync(anyLong(), anyLong());
+
+ // Initialize "entryCache.estimatedEntrySize" to the correct value.
+ Object ctx = new Object();
+ SimpleReadEntriesCallback cb0 = new SimpleReadEntriesCallback();
+ entryCache.asyncReadEntry(spyCurrentLedger, 125, 125, true, cb0, ctx);
+ cb0.entries.join();
+ Long sizePerEntry1 = entryCache.getEstimatedEntrySize();
+ Assert.assertEquals(sizePerEntry1, 1 + RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY);
+ Awaitility.await().untilAsserted(() -> {
+ long remainingBytes =limiter.getRemainingBytes();
+ Assert.assertEquals(remainingBytes, totalCapacity);
+ });
+ log.info("remainingBytes 0: {}", limiter.getRemainingBytes());
+
+ // Concurrency reading.
+
+ SimpleReadEntriesCallback cb1 = new SimpleReadEntriesCallback();
+ SimpleReadEntriesCallback cb2 = new SimpleReadEntriesCallback();
+ threadFactory.newThread(() -> {
+ entryCache.asyncReadEntry(spyCurrentLedger, start1, end1, true, cb1, ctx);
+ }).start();
+ threadFactory.newThread(() -> {
+ try {
+ firstReadingStarted.await();
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ entryCache.asyncReadEntry(spyCurrentLedger, start2, end2, true, cb2, ctx);
+ }).start();
+
+ long bytesAcquired1 = calculateBytesSizeBeforeFirstReading(readCount1 + readCount2, 1);
+ long remainingBytesExpected1 = totalCapacity - bytesAcquired1;
+ log.info("acquired : {}", bytesAcquired1);
+ log.info("remainingBytesExpected 0 : {}", remainingBytesExpected1);
+ Awaitility.await().untilAsserted(() -> {
+ log.info("remainingBytes 0: {}", limiter.getRemainingBytes());
+ Assert.assertEquals(limiter.getRemainingBytes(), remainingBytesExpected1);
+ });
+
+ // Complete the read1.
+ Thread.sleep(3000);
+ readCompleteSignal1.countDown();
+ cb1.entries.join();
+ Long sizePerEntry2 = entryCache.getEstimatedEntrySize();
+ Assert.assertEquals(sizePerEntry2, 1 + RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY);
+ long bytesAcquired2 = calculateBytesSizeBeforeFirstReading(readCount2, 1);
+ long remainingBytesExpected2 = totalCapacity - bytesAcquired2;
+ log.info("acquired : {}", bytesAcquired2);
+ log.info("remainingBytesExpected 1: {}", remainingBytesExpected2);
+ Awaitility.await().untilAsserted(() -> {
+ log.info("remainingBytes 1: {}", limiter.getRemainingBytes());
+ Assert.assertEquals(limiter.getRemainingBytes(), remainingBytesExpected2);
+ });
+
+ readCompleteSignal2.countDown();
+ cb2.entries.join();
+ Long sizePerEntry3 = entryCache.getEstimatedEntrySize();
+ Assert.assertEquals(sizePerEntry3, 1 + RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY);
+ Awaitility.await().untilAsserted(() -> {
+ long remainingBytes = limiter.getRemainingBytes();
+ log.info("remainingBytes 2: {}", remainingBytes);
+ Assert.assertEquals(remainingBytes, totalCapacity);
+ });
+ // cleanup
+ ml.delete();
+ factory.shutdown();
+ }
+
+ private long calculateBytesSizeBeforeFirstReading(int entriesCount, int perEntrySize) {
+ return entriesCount * (perEntrySize + RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY);
+ }
+
+ class SimpleReadEntriesCallback implements AsyncCallbacks.ReadEntriesCallback {
+
+ CompletableFuture> entries = new CompletableFuture<>();
+
+ @Override
+ public void readEntriesComplete(List entriesRead, Object ctx) {
+ List list = new ArrayList<>(entriesRead.size());
+ for (Entry entry : entriesRead) {
+ byte b = entry.getDataBuffer().readByte();
+ list.add(b);
+ entry.release();
+ }
+ this.entries.complete(list);
+ }
+
+ @Override
+ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
+ this.entries.completeExceptionally(exception);
+ }
+ }
+}
diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java
index 4c95454e33a92..fe484d62c4eff 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java
@@ -3222,7 +3222,7 @@ public void testOutOfOrderDeletePersistenceIntoLedgerWithClose() throws Exceptio
managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore(10);
ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerName, managedLedgerConfig);
- ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName);
+ final ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName);
List addedPositions = new ArrayList<>();
for (int i = 0; i < totalAddEntries; i++) {
@@ -3268,7 +3268,8 @@ public void operationFailed(MetaStoreException e) {
LedgerEntry entry = seq.nextElement();
PositionInfo positionInfo;
positionInfo = PositionInfo.parseFrom(entry.getEntry());
- individualDeletedMessagesCount.set(positionInfo.getIndividualDeletedMessagesCount());
+ c1.recoverIndividualDeletedMessages(positionInfo);
+ individualDeletedMessagesCount.set(c1.getIndividuallyDeletedMessagesSet().asRanges().size());
} catch (Exception e) {
}
latch.countDown();
@@ -3285,12 +3286,12 @@ public void operationFailed(MetaStoreException e) {
@Cleanup("shutdown")
ManagedLedgerFactory factory2 = new ManagedLedgerFactoryImpl(metadataStore, bkc);
ledger = (ManagedLedgerImpl) factory2.open(ledgerName, managedLedgerConfig);
- c1 = (ManagedCursorImpl) ledger.openCursor("c1");
+ ManagedCursorImpl reopenCursor = (ManagedCursorImpl) ledger.openCursor("c1");
// verify cursor has been recovered
- assertEquals(c1.getNumberOfEntriesInBacklog(false), totalAddEntries / 2);
+ assertEquals(reopenCursor.getNumberOfEntriesInBacklog(false), totalAddEntries / 2);
// try to read entries which should only read non-deleted positions
- List entries = c1.readEntries(totalAddEntries);
+ List entries = reopenCursor.readEntries(totalAddEntries);
assertEquals(entries.size(), totalAddEntries / 2);
}
diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java
index bb505200ba75e..783788be0d6d0 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java
@@ -20,24 +20,32 @@
import static org.apache.pulsar.common.util.PortManager.releaseLockedPort;
import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
+import com.fasterxml.jackson.databind.ObjectMapper;
import io.netty.buffer.ByteBuf;
-
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.Future;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import lombok.Cleanup;
+import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.BookKeeperTestClient;
+import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback;
import org.apache.bookkeeper.mledger.Entry;
@@ -50,14 +58,21 @@
import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager;
+import org.apache.bookkeeper.mledger.proto.MLDataFormats;
import org.apache.bookkeeper.mledger.util.ThrowableToStringUtil;
import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.awaitility.Awaitility;
+import org.testng.annotations.DataProvider;
import org.awaitility.Awaitility;
import org.testng.annotations.Test;
+@Slf4j
public class ManagedLedgerBkTest extends BookKeeperClusterTestCase {
+ private final ObjectMapper jackson = new ObjectMapper();
+
public ManagedLedgerBkTest() {
super(2);
}
@@ -229,6 +244,109 @@ public void verifyConcurrentUsage() throws Exception {
assertEquals(factory.getMbean().getNumberOfCacheEvictions(), 0);
}
+ @Test
+ public void verifyAsyncReadEntryUsingCache() throws Exception {
+ ManagedLedgerFactoryConfig config = new ManagedLedgerFactoryConfig();
+
+ config.setMaxCacheSize(100 * 1024 * 1024);
+ config.setCacheEvictionTimeThresholdMillis(10000);
+ config.setCacheEvictionIntervalMs(10000);
+
+ @Cleanup("shutdown")
+ ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, config);
+
+ ManagedLedgerConfig conf = new ManagedLedgerConfig();
+ conf.setEnsembleSize(2).setAckQuorumSize(2).setMetadataEnsembleSize(2)
+ .setRetentionSizeInMB(-1).setRetentionTime(-1, TimeUnit.MILLISECONDS);
+ final ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my-ledger" + testName, conf);
+
+ int NumProducers = 5;
+ int NumConsumers = 10;
+
+ final AtomicBoolean done = new AtomicBoolean();
+ final CyclicBarrier barrier = new CyclicBarrier(NumProducers + NumConsumers + 1);
+
+ List> futures = new ArrayList();
+ List positions = new CopyOnWriteArrayList<>();
+
+ for (int i = 0; i < NumProducers; i++) {
+ futures.add(executor.submit(() -> {
+ try {
+ // wait for all threads to be ready to start at once
+ barrier.await();
+ while (!done.get()) {
+ PositionImpl position = (PositionImpl) ledger.addEntry("entry".getBytes());
+ positions.add(position);
+ Thread.sleep(1);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw FutureUtil.wrapToCompletionException(e);
+ }
+ }));
+ }
+
+ // create a dummy cursor since caching happens only when there are active consumers
+ ManagedCursor cursor = ledger.openCursor("dummy");
+
+ for (int i = 0; i < NumConsumers; i++) {
+ futures.add(executor.submit(() -> {
+ try {
+ // wait for all threads to be ready to start at once
+ barrier.await();
+ while (!done.get()) {
+ if (positions.isEmpty()) {
+ Thread.sleep(1);
+ continue;
+ }
+ // Simulate a replay queue read pattern where individual entries are read
+ PositionImpl randomPosition =
+ positions.get(ThreadLocalRandom.current().nextInt(positions.size()));
+ // Clone the original instance so that another instance is used in the asyncReadEntry call
+ // This is to test that keys are compared by .equals and not by reference under the covers
+ randomPosition = PositionImpl.get(randomPosition);
+ CompletableFuture future = new CompletableFuture<>();
+ ledger.asyncReadEntry(randomPosition, new AsyncCallbacks.ReadEntryCallback() {
+ @Override
+ public void readEntryComplete(Entry entry, Object ctx) {
+ entry.release();
+ future.complete(null);
+ }
+
+ @Override
+ public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+ future.completeExceptionally(exception);
+ }
+ }, null);
+ future.get();
+ Thread.sleep(2);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw FutureUtil.wrapToCompletionException(e);
+ }
+ }));
+ }
+
+ // trigger all worker threads at once to continue from the barrier
+ barrier.await();
+
+ int testDurationSeconds = 3;
+ Thread.sleep(testDurationSeconds * 1000);
+
+ done.set(true);
+ for (Future> future : futures) {
+ future.get();
+ }
+
+ factory.getMbean().refreshStats(testDurationSeconds, TimeUnit.SECONDS);
+
+ assertTrue(factory.getMbean().getCacheHitsRate() > 0.0);
+ assertEquals(factory.getMbean().getCacheMissesRate(), 0.0);
+ assertTrue(factory.getMbean().getCacheHitsThroughput() > 0.0);
+ assertEquals(factory.getMbean().getNumberOfCacheEvictions(), 0);
+ }
+
@Test
public void testSimple() throws Exception {
@Cleanup("shutdown")
@@ -587,4 +705,114 @@ public void testPeriodicRollover() throws Exception {
Awaitility.await().until(() -> cursorImpl.getCursorLedger() != currentLedgerId);
}
+ @DataProvider(name = "unackedRangesOpenCacheSetEnabledPair")
+ public Object[][] unackedRangesOpenCacheSetEnabledPair() {
+ return new Object[][]{
+ {false, true},
+ {true, false},
+ {true, true},
+ {false, false}
+ };
+ }
+
+ /**
+ * This test validates that cursor serializes and deserializes individual-ack list from the bk-ledger.
+ * @throws Exception
+ */
+ @Test(dataProvider = "unackedRangesOpenCacheSetEnabledPair")
+ public void testUnackmessagesAndRecoveryCompatibility(boolean enabled1, boolean enabled2) throws Exception {
+ final String mlName = "ml" + UUID.randomUUID().toString().replaceAll("-", "");
+ final String cursorName = "c1";
+ ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig();
+ ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf);
+ final ManagedLedgerConfig config1 = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1)
+ .setAckQuorumSize(1).setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1)
+ .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1)
+ .setUnackedRangesOpenCacheSetEnabled(enabled1);
+ final ManagedLedgerConfig config2 = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1)
+ .setAckQuorumSize(1).setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1)
+ .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1)
+ .setUnackedRangesOpenCacheSetEnabled(enabled2);
+
+ ManagedLedger ledger1 = factory.open(mlName, config1);
+ ManagedCursorImpl cursor1 = (ManagedCursorImpl) ledger1.openCursor(cursorName);
+
+ int totalEntries = 100;
+ for (int i = 0; i < totalEntries; i++) {
+ Position p = ledger1.addEntry("entry".getBytes());
+ if (i % 2 == 0) {
+ cursor1.delete(p);
+ }
+ }
+ log.info("ack ranges: {}", cursor1.getIndividuallyDeletedMessagesSet().size());
+
+ // reopen and recover cursor
+ ledger1.close();
+ ManagedLedger ledger2 = factory.open(mlName, config2);
+ ManagedCursorImpl cursor2 = (ManagedCursorImpl) ledger2.openCursor(cursorName);
+
+ log.info("before: {}", cursor1.getIndividuallyDeletedMessagesSet().asRanges());
+ log.info("after : {}", cursor2.getIndividuallyDeletedMessagesSet().asRanges());
+ assertEquals(cursor1.getIndividuallyDeletedMessagesSet().asRanges(), cursor2.getIndividuallyDeletedMessagesSet().asRanges());
+ assertEquals(cursor1.markDeletePosition, cursor2.markDeletePosition);
+
+ ledger2.close();
+ factory.shutdown();
+ }
+
+ @DataProvider(name = "booleans")
+ public Object[][] booleans() {
+ return new Object[][] {
+ {true},
+ {false},
+ };
+ }
+
+ @Test(dataProvider = "booleans")
+ public void testConfigPersistIndividualAckAsLongArray(boolean enable) throws Exception {
+ final String mlName = "ml" + UUID.randomUUID().toString().replaceAll("-", "");
+ final String cursorName = "c1";
+ ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig();
+ ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf);
+ final ManagedLedgerConfig config = new ManagedLedgerConfig()
+ .setEnsembleSize(1).setWriteQuorumSize(1).setAckQuorumSize(1)
+ .setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1).setMetadataAckQuorumSize(1)
+ .setMaxUnackedRangesToPersistInMetadataStore(1)
+ .setUnackedRangesOpenCacheSetEnabled(true).setPersistIndividualAckAsLongArray(enable);
+
+ ManagedLedger ledger1 = factory.open(mlName, config);
+ ManagedCursorImpl cursor1 = (ManagedCursorImpl) ledger1.openCursor(cursorName);
+
+ // Write entries.
+ int totalEntries = 100;
+ List entries = new ArrayList<>();
+ for (int i = 0; i < totalEntries; i++) {
+ Position p = ledger1.addEntry("entry".getBytes());
+ entries.add(p);
+ }
+ // Make ack holes and trigger a mark deletion.
+ for (int i = totalEntries - 1; i >=0 ; i--) {
+ if (i % 2 == 0) {
+ cursor1.delete(entries.get(i));
+ }
+ }
+ cursor1.markDelete(entries.get(9));
+ Awaitility.await().untilAsserted(() -> {
+ assertEquals(cursor1.pendingMarkDeleteOps.size(), 0);
+ });
+
+ // Verify: the config affects.
+ long cursorLedgerLac = cursor1.cursorLedger.getLastAddConfirmed();
+ LedgerEntry ledgerEntry = cursor1.cursorLedger.readEntries(cursorLedgerLac, cursorLedgerLac).nextElement();
+ MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(ledgerEntry.getEntry());
+ if (enable) {
+ assertNotEquals(positionInfo.getIndividualDeletedMessageRangesList().size(), 0);
+ } else {
+ assertEquals(positionInfo.getIndividualDeletedMessageRangesList().size(), 0);
+ }
+
+ // cleanup
+ ledger1.close();
+ factory.shutdown();
+ }
}
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 8620604e41bf0..d09bb3ac3f58f 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
@@ -94,6 +94,7 @@
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.PulsarMockBookKeeper;
import org.apache.bookkeeper.client.PulsarMockLedgerHandle;
+import org.apache.bookkeeper.client.PulsarMockReadHandleInterceptor;
import org.apache.bookkeeper.client.api.LedgerEntries;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.client.api.ReadHandle;
@@ -3129,17 +3130,26 @@ public void testManagedLedgerWithReadEntryTimeOut() throws Exception {
ManagedLedgerConfig config = new ManagedLedgerConfig().setReadEntryTimeoutSeconds(1);
ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("timeout_ledger_test", config);
- BookKeeper bk = mock(BookKeeper.class);
- doNothing().when(bk).asyncCreateLedger(anyInt(), anyInt(), anyInt(), any(), any(), any(), any(), any());
+ Position position = ledger.addEntry("entry-1".getBytes());
+
+ // ensure that the read isn't cached
+ factory.getEntryCacheManager().clear();
+
+ bkc.setReadHandleInterceptor(new PulsarMockReadHandleInterceptor() {
+ @Override
+ public CompletableFuture interceptReadAsync(long ledgerId, long firstEntry, long lastEntry,
+ LedgerEntries entries) {
+ return CompletableFuture.supplyAsync(() -> {
+ return entries;
+ }, CompletableFuture.delayedExecutor(3, TimeUnit.SECONDS));
+ }
+ });
+
AtomicReference responseException1 = new AtomicReference<>();
String ctxStr = "timeoutCtx";
- CompletableFuture entriesFuture = new CompletableFuture<>();
- ReadHandle ledgerHandle = mock(ReadHandle.class);
- doReturn(entriesFuture).when(ledgerHandle).readAsync(PositionImpl.EARLIEST.getLedgerId(),
- PositionImpl.EARLIEST.getEntryId());
// (1) test read-timeout for: ManagedLedger.asyncReadEntry(..)
- ledger.asyncReadEntry(ledgerHandle, PositionImpl.EARLIEST, new ReadEntryCallback() {
+ ledger.asyncReadEntry((PositionImpl) position, new ReadEntryCallback() {
@Override
public void readEntryComplete(Entry entry, Object ctx) {
responseException1.set(null);
@@ -3151,18 +3161,20 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
responseException1.set(exception);
}
}, ctxStr);
- ledger.asyncCreateLedger(bk, config, null, (rc, lh, ctx) -> {}, Collections.emptyMap());
- retryStrategically((test) -> responseException1.get() != null, 5, 1000);
- assertNotNull(responseException1.get());
- assertTrue(responseException1.get().getMessage()
- .startsWith(BKException.getMessage(BKException.Code.TimeoutException)));
- // (2) test read-timeout for: ManagedLedger.asyncReadEntry(..)
- AtomicReference responseException2 = new AtomicReference<>();
- PositionImpl readPositionRef = PositionImpl.EARLIEST;
- ManagedCursorImpl cursor = new ManagedCursorImpl(bk, ledger, "cursor1");
- OpReadEntry opReadEntry = OpReadEntry.create(cursor, readPositionRef, 1, new ReadEntriesCallback() {
+ Awaitility.await().untilAsserted(() -> {
+ assertNotNull(responseException1.get());
+ assertTrue(responseException1.get().getMessage()
+ .startsWith(BKException.getMessage(BKException.Code.TimeoutException)));
+ });
+ // ensure that the read isn't cached
+ factory.getEntryCacheManager().clear();
+
+ // (2) test read-timeout for: ManagedCursor.asyncReadEntries(..)
+ AtomicReference responseException2 = new AtomicReference<>();
+ ManagedCursor cursor = ledger.openCursor("cursor1", InitialPosition.Earliest);
+ cursor.asyncReadEntries(1, new ReadEntriesCallback() {
@Override
public void readEntriesComplete(List entries, Object ctx) {
}
@@ -3172,16 +3184,13 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
assertEquals(ctxStr, (String) ctx);
responseException2.set(exception);
}
+ }, ctxStr, PositionImpl.LATEST);
- }, null, PositionImpl.LATEST, null);
- ledger.asyncReadEntry(ledgerHandle, PositionImpl.EARLIEST.getEntryId(), PositionImpl.EARLIEST.getEntryId(),
- opReadEntry, ctxStr);
- retryStrategically((test) -> {
- return responseException2.get() != null;
- }, 5, 1000);
- assertNotNull(responseException2.get());
- assertTrue(responseException2.get().getMessage()
- .startsWith(BKException.getMessage(BKException.Code.TimeoutException)));
+ Awaitility.await().untilAsserted(() -> {
+ assertNotNull(responseException2.get());
+ assertTrue(responseException2.get().getMessage()
+ .startsWith(BKException.getMessage(BKException.Code.TimeoutException)));
+ });
ledger.close();
}
@@ -3720,6 +3729,10 @@ public void testInvalidateReadHandleWhenDeleteLedger() throws Exception {
for (int i = 0; i < entries; i++) {
ledger.addEntry(String.valueOf(i).getBytes(Encoding));
}
+
+ // clear the cache to avoid flakiness
+ factory.getEntryCacheManager().clear();
+
List entryList = cursor.readEntries(3);
assertEquals(entryList.size(), 3);
Awaitility.await().untilAsserted(() -> {
@@ -3788,10 +3801,16 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception {
for (int i = 0; i < entries; i++) {
ledger.addEntry(String.valueOf(i).getBytes(Encoding));
}
- List entryList = cursor.readEntries(3);
- assertEquals(entryList.size(), 3);
- assertEquals(ledger.ledgers.size(), 4);
- assertEquals(ledger.ledgerCache.size(), 3);
+
+ // clear the cache to avoid flakiness
+ factory.getEntryCacheManager().clear();
+
+ final List entryList = cursor.readEntries(3);
+ Awaitility.await().untilAsserted(() -> {
+ assertEquals(entryList.size(), 3);
+ assertEquals(ledger.ledgers.size(), 4);
+ assertEquals(ledger.ledgerCache.size(), 3);
+ });
cursor.clearBacklog();
cursor2.clearBacklog();
ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE);
@@ -3800,11 +3819,17 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception {
assertEquals(ledger.ledgerCache.size(), 0);
});
+ // clear the cache to avoid flakiness
+ factory.getEntryCacheManager().clear();
+
// Verify the ReadHandle can be reopened.
ManagedCursor cursor3 = ledger.openCursor("test-cursor3", InitialPosition.Earliest);
- entryList = cursor3.readEntries(3);
- assertEquals(entryList.size(), 3);
- assertEquals(ledger.ledgerCache.size(), 3);
+ final List entryList2 = cursor3.readEntries(3);
+ Awaitility.await().untilAsserted(() -> {
+ assertEquals(entryList2.size(), 3);
+ assertEquals(ledger.ledgerCache.size(), 3);
+ });
+
cursor3.clearBacklog();
ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE);
Awaitility.await().untilAsserted(() -> {
@@ -3812,7 +3837,6 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception {
assertEquals(ledger.ledgerCache.size(), 0);
});
-
cursor.close();
cursor2.close();
cursor3.close();
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 2b69581ca2c73..3da8cdf517cbd 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
@@ -18,155 +18,492 @@
*/
package org.apache.bookkeeper.mledger.impl.cache;
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertFalse;
-import static org.testng.Assert.assertTrue;
-
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import java.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.atomic.AtomicReference;
+import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;
+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() {
+ return new Object[][]{
+ {0, true},
+ {-1, true},
+ {1, false},
+ };
+ }
- @Test
- public void testDisabled() throws Exception {
-
- InflightReadsLimiter limiter = new InflightReadsLimiter(0);
- assertTrue(limiter.isDisabled());
-
- limiter = new InflightReadsLimiter(-1);
- assertTrue(limiter.isDisabled());
+ @DataProvider
+ private static Object[] booleanValues() {
+ return new Object[]{ true, false };
+ }
- limiter = new InflightReadsLimiter(1);
- assertFalse(limiter.isDisabled());
+ @Test(dataProvider = "isDisabled")
+ public void testDisabled(long maxReadsInFlightSize, boolean shouldBeDisabled) throws Exception {
+ var limiter = new InflightReadsLimiter(maxReadsInFlightSize, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ mock(ScheduledExecutorService.class));
+ assertThat(limiter.isDisabled()).isEqualTo(shouldBeDisabled);
}
@Test
public void testBasicAcquireRelease() throws Exception {
- InflightReadsLimiter limiter = new InflightReadsLimiter(100);
- assertEquals(100, limiter.getRemainingBytes());
- InflightReadsLimiter.Handle handle = limiter.acquire(100, null);
- assertEquals(0, limiter.getRemainingBytes());
- assertTrue(handle.success);
- assertEquals(handle.acquiredPermits, 100);
- assertEquals(1, handle.trials);
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ mock(ScheduledExecutorService.class));
+ assertThat(limiter.getRemainingBytes()).isEqualTo(100);
+
+ Optional optionalHandle = limiter.acquire(100, null);
+ assertThat(limiter.getRemainingBytes()).isZero();
+ assertThat(optionalHandle).isPresent();
+ InflightReadsLimiter.Handle handle = optionalHandle.get();
+ assertThat(handle.success()).isTrue();
+ assertThat(handle.permits()).isEqualTo(100);
+
limiter.release(handle);
- assertEquals(100, limiter.getRemainingBytes());
+ assertThat(limiter.getRemainingBytes()).isEqualTo(100);
}
@Test
public void testNotEnoughPermits() throws Exception {
- InflightReadsLimiter limiter = new InflightReadsLimiter(100);
- assertEquals(100, limiter.getRemainingBytes());
- InflightReadsLimiter.Handle handle = limiter.acquire(100, null);
- assertEquals(0, limiter.getRemainingBytes());
- assertTrue(handle.success);
- assertEquals(handle.acquiredPermits, 100);
- assertEquals(1, handle.trials);
-
- InflightReadsLimiter.Handle handle2 = limiter.acquire(100, null);
- assertEquals(0, limiter.getRemainingBytes());
- assertFalse(handle2.success);
- assertEquals(handle2.acquiredPermits, 0);
- assertEquals(1, handle2.trials);
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ mock(ScheduledExecutorService.class));
+ assertThat(limiter.getRemainingBytes()).isEqualTo(100);
+ Optional optionalHandle = limiter.acquire(100, null);
+ assertThat(limiter.getRemainingBytes()).isZero();
+ assertThat(optionalHandle).isPresent();
+ InflightReadsLimiter.Handle handle = optionalHandle.get();
+ assertThat(handle.success()).isTrue();
+ assertThat(handle.permits()).isEqualTo(100);
+
+ AtomicReference handle2Reference = new AtomicReference<>();
+ Optional optionalHandle2 = limiter.acquire(100, handle2Reference::set);
+ assertThat(limiter.getRemainingBytes()).isZero();
+ assertThat(optionalHandle2).isNotPresent();
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());
+ assertThat(handle2Reference)
+ .hasValueSatisfying(h ->
+ assertThat(h.success()).isTrue());
+ limiter.release(handle2Reference.get());
+ assertThat(limiter.getRemainingBytes()).isEqualTo(100);
}
@Test
- public void testPartialAcquire() throws Exception {
- InflightReadsLimiter limiter = new InflightReadsLimiter(100);
- 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);
-
- limiter.release(handle2);
- assertEquals(100, limiter.getRemainingBytes());
-
+ public void testAcquireTimeout() throws Exception {
+ @Cleanup("shutdownNow")
+ ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ executor);
+ assertThat(limiter.getRemainingBytes()).isEqualTo(100);
+ limiter.acquire(100, null);
+
+ AtomicReference handle2Reference = new AtomicReference<>();
+ Optional optionalHandle2 = limiter.acquire(100, handle2Reference::set);
+ assertThat(optionalHandle2).isNotPresent();
+
+ Thread.sleep(ACQUIRE_TIMEOUT_MILLIS + 100);
+
+ assertThat(handle2Reference).hasValueSatisfying(h -> assertThat(h.success()).isFalse());
}
@Test
- public void testTooManyTrials() throws Exception {
- InflightReadsLimiter limiter = new InflightReadsLimiter(100);
- 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);
-
- 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);
-
- 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);
+ public void testMultipleQueuedEntriesWithExceptionInFirstCallback() throws Exception {
+ @Cleanup("shutdownNow")
+ ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ executor);
+ assertThat(limiter.getRemainingBytes())
+ .as("Initial remaining bytes should be 100")
+ .isEqualTo(100);
+
+ // Acquire the initial permits
+ Optional handle1 = limiter.acquire(100, null);
+ assertThat(handle1)
+ .as("Initial handle should be present")
+ .isPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be 0 after acquiring 100 permits")
+ .isEqualTo(0);
+
+ // Queue the first handle with a callback that throws an exception
+ AtomicReference handle2Reference = new AtomicReference<>();
+ Optional handle2 = limiter.acquire(50, handle -> {
+ handle2Reference.set(handle);
+ throw new RuntimeException("Callback exception");
+ });
+ assertThat(handle2)
+ .as("Second handle should not be present")
+ .isNotPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0 after failed acquisition")
+ .isEqualTo(0);
+
+ // Queue the second handle with a successful callback
+ AtomicReference handle3Reference = new AtomicReference<>();
+ Optional handle3 = limiter.acquire(50, handle3Reference::set);
+ assertThat(handle3)
+ .as("Third handle should not be present as queue is full")
+ .isNotPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0")
+ .isEqualTo(0);
+
+ // Release the initial handle to trigger the queued callbacks
+ limiter.release(handle1.get());
+
+ // Verify the first callback threw an exception but the second callback was handled successfully
+ assertThat(handle2Reference)
+ .as("Handle2 should have been set in the callback despite the exception")
+ .hasValueSatisfying(handle -> assertThat(handle.success())
+ .as("Handle2 should be marked as successful")
+ .isTrue());
+ assertThat(handle3Reference)
+ .as("Handle3 should have been set successfully")
+ .hasValueSatisfying(handle -> assertThat(handle.success())
+ .as("Handle3 should be marked as successful")
+ .isTrue());
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0 after first releases are acquired")
+ .isEqualTo(0);
+
+ // Release the second handle
+ limiter.release(handle3Reference.get());
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be 50 after releasing handle3")
+ .isEqualTo(50);
+
+ // Release the third handle
+ limiter.release(handle3Reference.get());
+ assertThat(limiter.getRemainingBytes())
+ .as("All bytes should be released, so remaining bytes should be 100")
+ .isEqualTo(100);
+ }
- limiter.release(handle);
+ @Test
+ public void testMultipleQueuedEntriesWithTimeoutAndExceptionInFirstCallback() throws Exception {
+ @Cleanup("shutdownNow")
+ ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ executor);
+ assertThat(limiter.getRemainingBytes())
+ .as("Initial remaining bytes should be 100")
+ .isEqualTo(100);
+
+ // Acquire the initial permits
+ Optional handle1 = limiter.acquire(100, null);
+ assertThat(handle1)
+ .as("The first handle should be present after acquiring 100 permits")
+ .isPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be 0 after acquiring all permits")
+ .isEqualTo(0);
+
+ // Queue the first handle with a callback that times out and throws an exception
+ AtomicReference handle2Reference = new AtomicReference<>();
+ Optional handle2 = limiter.acquire(50, handle -> {
+ handle2Reference.set(handle);
+ throw new RuntimeException("Callback exception on timeout");
+ });
+ assertThat(handle2)
+ .as("The second handle should not be present as the callback throws an exception")
+ .isNotPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0 after failed acquisition")
+ .isEqualTo(0);
+
+ // Introduce a delay to differentiate operations between queued entries
+ Thread.sleep(50);
+
+ // Queue the second handle with a successful callback
+ AtomicReference handle3Reference = new AtomicReference<>();
+ Optional handle3 = limiter.acquire(50, handle3Reference::set);
+ assertThat(handle3)
+ .as("The third handle should not be present as permits are still unavailable")
+ .isNotPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0 after failed acquisition attempt")
+ .isEqualTo(0);
+
+ // Wait for the timeout to occur
+ Thread.sleep(ACQUIRE_TIMEOUT_MILLIS + 100);
+
+ // Verify the first callback timed out and threw an exception, and the second callback was handled
+ assertThat(handle2Reference)
+ .as("Handle2 should have been set in the callback despite the exception")
+ .hasValueSatisfying(handle -> assertThat(handle.success())
+ .as("Handle2 should be marked as unsuccessful due to a timeout")
+ .isFalse());
+ assertThat(handle3Reference)
+ .as("Handle3 should have been set in the callback after the permits became available")
+ .hasValueSatisfying(handle -> assertThat(handle.success())
+ .as("Handle3 should be marked as unsuccessful due to a timeout")
+ .isFalse());
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0 as no permits were released")
+ .isEqualTo(0);
+
+ // Release the first handle
+ limiter.release(handle1.get());
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be fully restored to 100 after releasing all permits")
+ .isEqualTo(100);
+ }
- handle2 = limiter.acquire(100, handle2);
- assertEquals(0, limiter.getRemainingBytes());
- assertTrue(handle2.success);
- assertEquals(handle2.acquiredPermits, 100);
- assertEquals(2, handle2.trials);
+ @Test
+ public void testMultipleQueuedEntriesWithTimeoutsThatAreTimedOutWhenPermitsAreAvailable() throws Exception {
+ // Use a mock executor to simulate scenarios where timed out queued handles are processed when permits become
+ // available
+ ScheduledExecutorService executor = mock(ScheduledExecutorService.class);
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS,
+ executor);
+ assertThat(limiter.getRemainingBytes())
+ .as("Initial remaining bytes should be 100")
+ .isEqualTo(100);
+
+ // Acquire the initial permits
+ Optional handle1 = limiter.acquire(100, null);
+ assertThat(handle1)
+ .as("The first handle should be present after acquiring 100 permits")
+ .isPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be 0 after acquiring all permits")
+ .isEqualTo(0);
+
+ // Queue the first handle
+ AtomicReference handle2Reference = new AtomicReference<>();
+ Optional handle2 = limiter.acquire(50, handle2Reference::set);
+ assertThat(handle2)
+ .as("The second handle should not be present as permits are unavailable")
+ .isNotPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0 after failed acquisition attempt for handle2")
+ .isEqualTo(0);
+
+ // Queue the second handle
+ AtomicReference handle3Reference = new AtomicReference<>();
+ Optional handle3 = limiter.acquire(50, handle3Reference::set);
+ assertThat(handle3)
+ .as("The third handle should not be present as permits are unavailable")
+ .isNotPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0 after failed acquisition attempt for handle3")
+ .isEqualTo(0);
+
+ // Wait for the timeout to occur
+ Thread.sleep(ACQUIRE_TIMEOUT_MILLIS + 100);
+
+ // Queue another handle
+ AtomicReference handle4Reference = new AtomicReference<>();
+ Optional handle4 = limiter.acquire(50, handle4Reference::set);
+ assertThat(handle4)
+ .as("The fourth handle should not be present because permits are unavailable")
+ .isNotPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0 after failed acquisition attempt for handle4")
+ .isEqualTo(0);
+
+ // Queue another handle
+ AtomicReference handle5Reference = new AtomicReference<>();
+ Optional handle5 = limiter.acquire(100, handle5Reference::set);
+ assertThat(handle5)
+ .as("The fifth handle should not be present as permits are unavailable")
+ .isNotPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should still be 0 after failed acquisition attempt for handle5")
+ .isEqualTo(0);
+
+ // Release the first handle
+ limiter.release(handle1.get());
+
+ assertThat(handle2Reference)
+ .as("Handle2 should have been set in the callback and marked unsuccessful")
+ .hasValueSatisfying(handle -> assertThat(handle.success()).isFalse());
+
+ assertThat(handle3Reference)
+ .as("Handle3 should have been set in the callback and marked unsuccessful")
+ .hasValueSatisfying(handle -> assertThat(handle.success()).isFalse());
+
+ assertThat(handle4Reference)
+ .as("Handle4 should have been set in the callback and marked successful")
+ .hasValueSatisfying(handle -> assertThat(handle.success()).isTrue());
+
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be 50 after releasing handle4")
+ .isEqualTo(50);
+
+ limiter.release(handle4Reference.get());
+
+ assertThat(handle5Reference)
+ .as("Handle5 should have been set in the callback and marked successful")
+ .hasValueSatisfying(handle -> assertThat(handle.success()).isTrue());
+
+ limiter.release(handle5Reference.get());
+
+ assertThat(limiter.getRemainingBytes())
+ .as("All bytes should be released, so remaining bytes should be back to 100")
+ .isEqualTo(100);
+ }
- limiter.release(handle2);
- assertEquals(100, limiter.getRemainingBytes());
+ @Test
+ public void testQueueSizeLimitReached() throws Exception {
+ @Cleanup("shutdownNow")
+ ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+
+ // Minimum queue size is 4.
+ final int queueSizeLimit = 4;
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(100, queueSizeLimit, ACQUIRE_TIMEOUT_MILLIS, executor);
+
+ assertThat(limiter.getRemainingBytes())
+ .as("Initial remaining bytes should be 100")
+ .isEqualTo(100);
+
+ // Acquire all available permits (consume 100 bytes)
+ Optional handle1 = limiter.acquire(100, null);
+ assertThat(handle1)
+ .as("The first handle should be present after acquiring all available permits")
+ .isPresent()
+ .hasValueSatisfying(handle -> assertThat(handle.success()).isTrue());
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be zero after acquiring all permits")
+ .isEqualTo(0);
+
+ // Queue up to the limit (4 requests)
+ AtomicReference handle2Reference = new AtomicReference<>();
+ assertThat(limiter.acquire(50, handle2Reference::set)).isNotPresent();
+
+ AtomicReference handle3Reference = new AtomicReference<>();
+ assertThat(limiter.acquire(50, handle3Reference::set)).isNotPresent();
+
+ AtomicReference handle4Reference = new AtomicReference<>();
+ assertThat(limiter.acquire(50, handle4Reference::set)).isNotPresent();
+
+ AtomicReference handle5Reference = new AtomicReference<>();
+ assertThat(limiter.acquire(50, handle5Reference::set)).isNotPresent();
+
+ // Attempt to add one more request, which should fail as the queue is full
+ Optional handle6 = limiter.acquire(50, null);
+ assertThat(handle6)
+ .as("The sixth handle should not be successfull since the queue is full")
+ .hasValueSatisfying(handle -> assertThat(handle.success()).isFalse());
+ }
+ @Test(dataProvider = "booleanValues")
+ public void testAcquireExceedingMaxReadsInFlightSize(boolean firstInQueue) throws Exception {
+ @Cleanup("shutdownNow")
+ ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+
+ long maxReadsInFlightSize = 100;
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(maxReadsInFlightSize, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, executor);
+
+ // Initial state
+ assertThat(limiter.getRemainingBytes())
+ .as("Initial remaining bytes should match maxReadsInFlightSize")
+ .isEqualTo(maxReadsInFlightSize);
+
+ // Acquire all permits (consume 100 bytes)
+ Optional handle1 = limiter.acquire(100, null);
+ assertThat(handle1)
+ .as("The first handle should be present")
+ .isPresent();
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be zero after acquiring all permits")
+ .isEqualTo(0);
+
+
+ AtomicReference handle2Reference = new AtomicReference<>();
+
+ if (!firstInQueue) {
+ Optional handle2 = limiter.acquire(50, handle2Reference::set);
+ assertThat(handle2)
+ .as("The second handle should not be present as remaining permits are zero")
+ .isNotPresent();
+ }
+
+ // Attempt to acquire more than maxReadsInFlightSize while all permits are in use
+ AtomicReference handleExceedingMaxReference = new AtomicReference<>();
+ Optional handleExceedingMaxOptional =
+ limiter.acquire(200, handleExceedingMaxReference::set);
+ assertThat(handleExceedingMaxOptional)
+ .as("The second handle should not be present as remaining permits are zero")
+ .isNotPresent();
+
+ // Release handle1 permits
+ limiter.release(handle1.get());
+
+ if (!firstInQueue) {
+ assertThat(handle2Reference)
+ .as("Handle2 should have been set in the callback and marked successful")
+ .hasValueSatisfying(handle -> {
+ assertThat(handle.success()).isTrue();
+ assertThat(handle.permits()).isEqualTo(50);
+ });
+ limiter.release(handle2Reference.get());
+ }
+
+ assertThat(handleExceedingMaxReference)
+ .as("Handle2 should have been set in the callback and marked successful")
+ .hasValueSatisfying(handle -> {
+ assertThat(handle.success()).isTrue();
+ assertThat(handle.permits()).isEqualTo(maxReadsInFlightSize);
+ });
+
+ limiter.release(handleExceedingMaxReference.get());
+
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be fully replenished after releasing all permits")
+ .isEqualTo(maxReadsInFlightSize);
}
+ @Test
+ public void testAcquireExceedingMaxReadsWhenAllPermitsAvailable() throws Exception {
+ @Cleanup("shutdownNow")
+ ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+
+ long maxReadsInFlightSize = 100;
+ InflightReadsLimiter limiter =
+ new InflightReadsLimiter(maxReadsInFlightSize, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, executor);
+
+ // Initial state
+ assertThat(limiter.getRemainingBytes())
+ .as("Initial remaining bytes should match maxReadsInFlightSize")
+ .isEqualTo(maxReadsInFlightSize);
+
+ // Acquire permits > maxReadsInFlightSize
+ Optional handleExceedingMaxOptional =
+ limiter.acquire(2 * maxReadsInFlightSize, null);
+ assertThat(handleExceedingMaxOptional)
+ .as("The handle for exceeding max permits should be present")
+ .hasValueSatisfying(handle -> {
+ assertThat(handle.success()).isTrue();
+ assertThat(handle.permits()).isEqualTo(maxReadsInFlightSize);
+ });
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be zero after acquiring all permits")
+ .isEqualTo(0);
+
+ // Release permits
+ limiter.release(handleExceedingMaxOptional.get());
+
+ assertThat(limiter.getRemainingBytes())
+ .as("Remaining bytes should be fully replenished after releasing all permits")
+ .isEqualTo(maxReadsInFlightSize);
+ }
}
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 6f573ff8d75c8..ea4c7189933d5 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
@@ -20,6 +20,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
import lombok.Data;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.client.api.ReadHandle;
@@ -93,7 +94,8 @@ void setupMocks() {
config.setReadEntryTimeoutSeconds(10000);
when(rangeEntryCache.getName()).thenReturn("my-topic");
when(rangeEntryCache.getManagedLedgerConfig()).thenReturn(config);
- inflighReadsLimiter = new InflightReadsLimiter(0);
+ inflighReadsLimiter = new InflightReadsLimiter(0, 0, 0,
+ mock(ScheduledExecutorService.class));
when(rangeEntryCache.getPendingReadsLimiter()).thenReturn(inflighReadsLimiter);
pendingReadsManager = new PendingReadsManager(rangeEntryCache);
doAnswer(new Answer() {
@@ -110,7 +112,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
return null;
}
}).when(rangeEntryCache).asyncReadEntry0(any(), anyLong(), anyLong(),
- anyBoolean(), any(), any());
+ anyBoolean(), any(), any(), anyBoolean());
lh = mock(ReadHandle.class);
ml = mock(ManagedLedgerImpl.class);
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
index 4bcf2cc6c4e35..aa13d4b8e3488 100644
--- 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
@@ -20,6 +20,7 @@
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotSame;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
@@ -338,4 +339,20 @@ public void testRemoveEntryWithInvalidMatchingKey() {
cache.clear();
assertEquals(cache.getNumberOfEntries(), 0);
}
-}
+
+ @Test
+ public void testGetKeyWithDifferentInstance() {
+ RangeCache cache = new RangeCache<>();
+ Integer key = 129;
+ cache.put(key, new RefString("129"));
+ // create a different instance of the key
+ Integer key2 = Integer.valueOf(129);
+ // key and key2 are different instances but they are equal
+ assertNotSame(key, key2);
+ assertEquals(key, key2);
+ // get the value using key2
+ RefString s = cache.get(key2);
+ // the value should be found
+ assertEquals(s.s, "129");
+ }
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index eae70d5d3f1be..5b679ab5f46ee 100644
--- a/pom.xml
+++ b/pom.xml
@@ -143,7 +143,7 @@ flexible messaging model and an intuitive client API.
1.1.10.5
4.1.12.1
5.7.1
- 4.1.116.Final
+ 4.1.117.Final
0.0.26.Final
9.4.56.v20240826
2.5.2
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
index 9e92d3664fd11..44086ca9130e1 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
@@ -2022,6 +2022,15 @@ The delayed message index time step(in seconds) in per bucket snapshot segment,
+ " Consumer Netty channel. Use O to disable")
private long managedLedgerMaxReadsInFlightSizeInMB = 0;
+ @FieldContext(category = CATEGORY_STORAGE_ML, doc = "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;
+
+ @FieldContext(category = CATEGORY_STORAGE_ML, doc = "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 = 50000;
+
@FieldContext(
category = CATEGORY_STORAGE_ML,
dynamic = true,
@@ -2146,6 +2155,10 @@ The delayed message index time step(in seconds) in per bucket snapshot segment,
+ " will only be tracked in memory and messages will be redelivered in case of"
+ " crashes.")
private int managedLedgerMaxUnackedRangesToPersist = 10000;
+ @FieldContext(
+ category = CATEGORY_STORAGE_ML,
+ doc = "Whether persist cursor ack stats as long arrays, which will compress the data and reduce GC rate")
+ private boolean managedLedgerPersistIndividualAckAsLongArray = false;
@FieldContext(
category = CATEGORY_STORAGE_ML,
doc = "If enabled, the maximum \"acknowledgment holes\" will not be limited and \"acknowledgment holes\" "
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java
index 22296b86b4e0c..1056cfc8b280a 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java
@@ -23,11 +23,10 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
-import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import javax.naming.AuthenticationException;
import javax.servlet.http.HttpServletRequest;
@@ -48,13 +47,13 @@ public class AuthenticationService implements Closeable {
private static final Logger LOG = LoggerFactory.getLogger(AuthenticationService.class);
private final String anonymousUserRole;
- private final Map providers = new HashMap<>();
+ private final Map providers = new LinkedHashMap<>();
public AuthenticationService(ServiceConfiguration conf) throws PulsarServerException {
anonymousUserRole = conf.getAnonymousUserRole();
if (conf.isAuthenticationEnabled()) {
try {
- Map> providerMap = new HashMap<>();
+ Map> providerMap = new LinkedHashMap<>();
for (String className : conf.getAuthenticationProviders()) {
if (className.isEmpty()) {
continue;
@@ -121,7 +120,7 @@ public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletRe
AuthenticationProvider providerToUse = getAuthProvider(authMethodName);
try {
return providerToUse.authenticateHttpRequest(request, response);
- } catch (AuthenticationException e) {
+ } catch (Exception e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Authentication failed for provider " + providerToUse.getAuthMethodName() + " : "
+ e.getMessage(), e);
@@ -132,7 +131,7 @@ public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletRe
for (AuthenticationProvider provider : providers.values()) {
try {
return provider.authenticateHttpRequest(request, response);
- } catch (AuthenticationException e) {
+ } catch (Exception e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Authentication failed for provider " + provider.getAuthMethodName() + ": "
+ e.getMessage(), e);
@@ -173,25 +172,18 @@ public String authenticateHttpRequest(HttpServletRequest request, Authentication
}
// Backward compatible, the authData value was null in the previous implementation
return providerToUse.authenticateAsync(authData).get();
- } catch (AuthenticationException e) {
+ } catch (Exception e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Authentication failed for provider " + providerToUse.getAuthMethodName() + " : "
+ e.getMessage(), e);
}
- throw e;
- } catch (ExecutionException | InterruptedException e) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Authentication failed for provider " + providerToUse.getAuthMethodName() + " : "
- + e.getMessage(), e);
- }
- throw new RuntimeException(e);
}
} else {
for (AuthenticationProvider provider : providers.values()) {
try {
AuthenticationState authenticationState = provider.newHttpAuthState(request);
return provider.authenticateAsync(authenticationState.getAuthDataSource()).get();
- } catch (ExecutionException | InterruptedException | AuthenticationException e) {
+ } catch (Exception e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Authentication failed for provider " + provider.getAuthMethodName() + ": "
+ e.getMessage(), e);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java
index 51fb8bc1ae38a..25e4a7c9c442e 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java
@@ -63,8 +63,21 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata
managedLedgerFactoryConfig.setCacheEvictionTimeThresholdMillis(
conf.getManagedLedgerCacheEvictionTimeThresholdMillis());
managedLedgerFactoryConfig.setCopyEntriesInCache(conf.isManagedLedgerCacheCopyEntries());
- managedLedgerFactoryConfig.setManagedLedgerMaxReadsInFlightSize(
- conf.getManagedLedgerMaxReadsInFlightSizeInMB() * 1024L * 1024L);
+ long managedLedgerMaxReadsInFlightSizeBytes = conf.getManagedLedgerMaxReadsInFlightSizeInMB() * 1024L * 1024L;
+ if (managedLedgerMaxReadsInFlightSizeBytes > 0 && conf.getDispatcherMaxReadSizeBytes() > 0
+ && managedLedgerMaxReadsInFlightSizeBytes < conf.getDispatcherMaxReadSizeBytes()) {
+ log.warn("Invalid configuration for managedLedgerMaxReadsInFlightSizeInMB: {}, "
+ + "dispatcherMaxReadSizeBytes: {}. managedLedgerMaxReadsInFlightSizeInMB in bytes should "
+ + "be greater than dispatcherMaxReadSizeBytes. You should set "
+ + "managedLedgerMaxReadsInFlightSizeInMB to at least {}",
+ conf.getManagedLedgerMaxReadsInFlightSizeInMB(), conf.getDispatcherMaxReadSizeBytes(),
+ (conf.getDispatcherMaxReadSizeBytes() / (1024L * 1024L)) + 1);
+ }
+ managedLedgerFactoryConfig.setManagedLedgerMaxReadsInFlightSize(managedLedgerMaxReadsInFlightSizeBytes);
+ managedLedgerFactoryConfig.setManagedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis(
+ conf.getManagedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis());
+ managedLedgerFactoryConfig.setManagedLedgerMaxReadsInFlightPermitsAcquireQueueSize(
+ conf.getManagedLedgerMaxReadsInFlightPermitsAcquireQueueSize());
managedLedgerFactoryConfig.setPrometheusStatsLatencyRolloverSeconds(
conf.getManagedLedgerPrometheusStatsLatencyRolloverSeconds());
managedLedgerFactoryConfig.setTraceTaskExecution(conf.isManagedLedgerTraceTaskExecution());
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
index 6e65830330f6f..138727a7be9cc 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
@@ -1530,23 +1530,21 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean
}
}
if (perPartition && stats.partitions.isEmpty()) {
- try {
- boolean pathExists = namespaceResources().getPartitionedTopicResources()
- .partitionedTopicExists(topicName);
- if (pathExists) {
- stats.partitions.put(topicName.toString(), new TopicStatsImpl());
- } else {
- asyncResponse.resume(
- new RestException(Status.NOT_FOUND,
- "Internal topics have not been generated yet"));
- return null;
- }
- } catch (Exception e) {
- asyncResponse.resume(new RestException(e));
- return null;
- }
+ namespaceResources().getPartitionedTopicResources()
+ .partitionedTopicExistsAsync(topicName)
+ .thenAccept(exists -> {
+ if (exists) {
+ stats.partitions.put(topicName.toString(), new TopicStatsImpl());
+ asyncResponse.resume(stats);
+ } else {
+ asyncResponse.resume(
+ new RestException(Status.NOT_FOUND,
+ "Internal topics have not been generated yet"));
+ }
+ });
+ } else {
+ asyncResponse.resume(stats);
}
- asyncResponse.resume(stats);
return null;
});
}).exceptionally(ex -> {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java
index 9fc6b9581a3ac..60908a013c1ea 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java
@@ -71,6 +71,10 @@ public SubType getType() {
public abstract boolean isConsumerAvailable(Consumer consumer);
+ /**
+ * Cancel a possible pending read that is a Managed Cursor waiting to be notified for more entries.
+ * This won't cancel any other pending reads that are currently in progress.
+ */
protected void cancelPendingRead() {}
/**
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
index 2236018aab5e1..ca53ab0fc3183 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
@@ -1948,6 +1948,8 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T
managedLedgerConfig
.setMaxUnackedRangesToPersist(serviceConfig.getManagedLedgerMaxUnackedRangesToPersist());
+ managedLedgerConfig
+ .setPersistIndividualAckAsLongArray(serviceConfig.isManagedLedgerPersistIndividualAckAsLongArray());
managedLedgerConfig.setPersistentUnackedRangesWithMultipleEntriesEnabled(
serviceConfig.isPersistentUnackedRangesWithMultipleEntriesEnabled());
managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore(
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java
index 183c6040205bc..80db47eda1c5d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java
@@ -958,6 +958,9 @@ public CompletableFuture checkPermissionsAsync() {
@Override
public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
if (obj instanceof Consumer) {
Consumer other = (Consumer) obj;
return consumerId == other.consumerId && Objects.equals(cnx.clientAddress(), other.cnx.clientAddress());
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
index 9e8483be1a701..ae844b5784456 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
@@ -553,9 +553,8 @@ public synchronized CompletableFuture disconnectAllConsumers(boolean isRes
@Override
protected void cancelPendingRead() {
- if ((havePendingRead || havePendingReplayRead) && cursor.cancelPendingReadRequest()) {
+ if (havePendingRead && cursor.cancelPendingReadRequest()) {
havePendingRead = false;
- havePendingReplayRead = false;
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
index 2fd0347c41f7b..0f6a6db2cd299 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
@@ -3243,8 +3243,8 @@ public CompletableFuture updateOldPositionInfo() {
TopicName topicName = TopicName.get(getName());
if (!(ledger.getCursors() instanceof ManagedCursorContainer managedCursorContainer)) {
- return CompletableFuture.failedFuture(new IllegalStateException(
- String.format("[%s] No valid cursors found. Skip update old position info.", topicName)));
+ // TODO: support this method with a customized managed ledger implementation
+ return CompletableFuture.completedFuture(null);
}
if (!hasBacklogs(brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck())) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java
index 7f0dac6fee4b6..1d7d8a672e85f 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java
@@ -341,7 +341,7 @@ public static void printTopicStats(PrometheusMetricStreams stream, TopicStats st
writeSubscriptionMetric(stream, "pulsar_subscription_filter_rescheduled_msg_count",
subsStats.filterRescheduledMsgCount, cluster, namespace, topic, sub,
splitTopicAndPartitionIndexLabel);
- writeSubscriptionMetric(stream, "pulsar_delayed_message_index_size_bytes",
+ writeSubscriptionMetric(stream, "pulsar_subscription_delayed_message_index_size_bytes",
subsStats.delayedMessageIndexSizeInBytes, cluster, namespace, topic, sub,
splitTopicAndPartitionIndexLabel);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java
index e42cae2580b78..24d0924b726a4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java
@@ -18,7 +18,6 @@
*/
package org.apache.pulsar.utils;
-import java.util.Iterator;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
@@ -27,8 +26,13 @@
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.pulsar.common.util.collections.LongPairSet;
+import org.roaringbitmap.PeekableIntIterator;
import org.roaringbitmap.RoaringBitmap;
+/**
+ * A concurrent set of pairs of longs.
+ * The right side of the value supports unsigned values up to 2^32.
+ */
public class ConcurrentBitmapSortedLongPairSet {
private final NavigableMap map = new TreeMap<>();
@@ -100,9 +104,11 @@ public > NavigableSet items(int numberOfItems,
lock.readLock().lock();
try {
for (Map.Entry entry : map.entrySet()) {
- Iterator iterator = entry.getValue().stream().iterator();
- while (iterator.hasNext() && items.size() < numberOfItems) {
- items.add(longPairConverter.apply(entry.getKey(), iterator.next()));
+ PeekableIntIterator intIterator = entry.getValue().getIntIterator();
+ while (intIterator.hasNext() && items.size() < numberOfItems) {
+ // RoaringBitmap encodes values as unsigned 32-bit integers internally, it's necessary to use
+ // Integer.toUnsignedLong to convert them to unsigned long values
+ items.add(longPairConverter.apply(entry.getKey(), Integer.toUnsignedLong(intIterator.next())));
}
if (items.size() == numberOfItems) {
break;
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java
index bfb172d0711d4..e97928c4c66e0 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java
@@ -18,9 +18,42 @@
*/
package org.apache.pulsar.broker;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.time.Duration;
+import java.util.Arrays;
import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+import java.util.stream.Stream;
+import lombok.SneakyThrows;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.ObjectMapperFactory;
import org.mockito.Mockito;
-
+import org.slf4j.Logger;
/**
* Holds util methods used in test.
*/
@@ -77,4 +110,268 @@ public static T spyWithoutRecordingInvocations(T object) {
.defaultAnswer(Mockito.CALLS_REAL_METHODS)
.stubOnly());
}
+
+ /**
+ * Uses Jackson to create a JSON string for the given object
+ * @param object to convert to JSON
+ * @return JSON string
+ */
+ public static String toJson(Object object) {
+ ObjectWriter writer = ObjectMapperFactory.getMapper().writer();
+ StringWriter stringWriter = new StringWriter();
+ try (JsonGenerator generator = writer.createGenerator(stringWriter).useDefaultPrettyPrinter()) {
+ generator.writeObject(object);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ return stringWriter.toString();
+ }
+
+ /**
+ * Logs the topic stats and internal stats for the given topic
+ * @param logger logger to use
+ * @param pulsarAdmin PulsarAdmin client to use
+ * @param topic topic name
+ */
+ public static void logTopicStats(Logger logger, PulsarAdmin pulsarAdmin, String topic) {
+ try {
+ logger.info("[{}] stats: {}", topic, toJson(pulsarAdmin.topics().getStats(topic)));
+ logger.info("[{}] internalStats: {}", topic,
+ toJson(pulsarAdmin.topics().getInternalStats(topic, true)));
+ } catch (PulsarAdminException e) {
+ logger.warn("Failed to get stats for topic {}", topic, e);
+ }
+ }
+
+ /**
+ * Logs the topic stats and internal stats for the given topic
+ * @param logger logger to use
+ * @param baseUrl Pulsar service URL
+ * @param topic topic name
+ */
+ public static void logTopicStats(Logger logger, String baseUrl, String topic) {
+ logTopicStats(logger, baseUrl, "public", "default", topic);
+ }
+
+ /**
+ * Logs the topic stats and internal stats for the given topic
+ * @param logger logger to use
+ * @param baseUrl Pulsar service URL
+ * @param tenant tenant name
+ * @param namespace namespace name
+ * @param topic topic name
+ */
+ public static void logTopicStats(Logger logger, String baseUrl, String tenant, String namespace, String topic) {
+ String topicStatsUri =
+ String.format("%s/admin/v2/persistent/%s/%s/%s/stats", baseUrl, tenant, namespace, topic);
+ logger.info("[{}] stats: {}", topic, jsonPrettyPrint(getJsonResourceAsString(topicStatsUri)));
+ String topicStatsInternalUri =
+ String.format("%s/admin/v2/persistent/%s/%s/%s/internalStats", baseUrl, tenant, namespace, topic);
+ logger.info("[{}] internalStats: {}", topic, jsonPrettyPrint(getJsonResourceAsString(topicStatsInternalUri)));
+ }
+
+ /**
+ * Pretty print the given JSON string
+ * @param jsonString JSON string to pretty print
+ * @return pretty printed JSON string
+ */
+ public static String jsonPrettyPrint(String jsonString) {
+ try {
+ ObjectMapper mapper = new ObjectMapper();
+ Object json = mapper.readValue(jsonString, Object.class);
+ ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
+ return writer.writeValueAsString(json);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ /**
+ * Get the resource as a string from the given URI
+ */
+ @SneakyThrows
+ public static String getJsonResourceAsString(String uri) {
+ URL url = new URL(uri);
+ HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+ connection.setRequestMethod("GET");
+ connection.setRequestProperty("Accept", "application/json");
+ try {
+ int responseCode = connection.getResponseCode();
+ if (responseCode == 200) {
+ try (BufferedReader in = new BufferedReader(new InputStreamReader(connection.getInputStream()))) {
+ String inputLine;
+ StringBuilder content = new StringBuilder();
+ while ((inputLine = in.readLine()) != null) {
+ content.append(inputLine);
+ }
+ return content.toString();
+ }
+ } else {
+ throw new IOException("Failed to get resource: " + uri + ", status: " + responseCode);
+ }
+ } finally {
+ connection.disconnect();
+ }
+ }
+
+ /**
+ * Receive messages concurrently from multiple consumers and handles them using the provided message handler.
+ * The message handler should return true if it wants to continue receiving more messages, false otherwise.
+ *
+ * @param messageHandler the message handler
+ * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages
+ * @param consumers the consumers to receive messages from
+ * @param the message value type
+ */
+ public static void receiveMessages(BiFunction, Message, Boolean> messageHandler,
+ Duration quietTimeout,
+ Consumer... consumers) {
+ receiveMessages(messageHandler, quietTimeout, Arrays.stream(consumers));
+ }
+
+ /**
+ * Receive messages concurrently from multiple consumers and handles them using the provided message handler.
+ * The message handler should return true if it wants to continue receiving more messages, false otherwise.
+ *
+ * @param messageHandler the message handler
+ * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages
+ * @param consumers the consumers to receive messages from
+ * @param the message value type
+ */
+ public static void receiveMessages(BiFunction, Message, Boolean> messageHandler,
+ Duration quietTimeout,
+ Stream> consumers) {
+ long quietTimeoutNanos = quietTimeout.toNanos();
+ AtomicLong lastMessageReceivedNanos = new AtomicLong(System.nanoTime());
+ FutureUtil.waitForAll(consumers
+ .map(consumer -> receiveMessagesAsync(consumer, quietTimeoutNanos, quietTimeoutNanos, messageHandler,
+ lastMessageReceivedNanos)).toList()).join();
+ }
+
+ // asynchronously receive messages from a consumer and handle them using the provided message handler
+ // the benefit is that multiple consumers can be concurrently consumed without the need to have multiple threads
+ // this is useful in tests where multiple consumers are needed to test the functionality
+ private static CompletableFuture receiveMessagesAsync(Consumer consumer,
+ long quietTimeoutNanos,
+ long receiveTimeoutNanos,
+ BiFunction, Message, Boolean>
+ messageHandler,
+ AtomicLong lastMessageReceivedNanos) {
+ return consumer.receiveAsync()
+ .orTimeout(receiveTimeoutNanos, TimeUnit.NANOSECONDS)
+ .handle((msg, t) -> {
+ long currentNanos = System.nanoTime();
+ if (t != null) {
+ if (t instanceof TimeoutException) {
+ long sinceLastMessageReceivedNanos = currentNanos - lastMessageReceivedNanos.get();
+ if (sinceLastMessageReceivedNanos > quietTimeoutNanos) {
+ return Pair.of(false, 0L);
+ } else {
+ return Pair.of(true, quietTimeoutNanos - sinceLastMessageReceivedNanos);
+ }
+ } else {
+ throw FutureUtil.wrapToCompletionException(t);
+ }
+ }
+ lastMessageReceivedNanos.set(currentNanos);
+ return Pair.of(messageHandler.apply(consumer, msg), quietTimeoutNanos);
+ }).thenComposeAsync(receiveMoreAndNextTimeout -> {
+ boolean receiveMore = receiveMoreAndNextTimeout.getLeft();
+ if (receiveMore) {
+ Long nextReceiveTimeoutNanos = receiveMoreAndNextTimeout.getRight();
+ return receiveMessagesAsync(consumer, quietTimeoutNanos, nextReceiveTimeoutNanos,
+ messageHandler, lastMessageReceivedNanos);
+ } else {
+ return CompletableFuture.completedFuture(null);
+ }
+ });
+ }
+
+ /**
+ * Receive messages concurrently from multiple consumers and handles them using the provided message handler.
+ * The messages are received until the quiet timeout is reached or the maximum number of messages is received.
+ *
+ * @param messageHandler the message handler
+ * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages
+ * @param maxMessages the maximum number of messages to receive
+ * @param consumers the consumers to receive messages from
+ * @param the message value type
+ */
+ public static void receiveMessagesN(BiConsumer, Message> messageHandler,
+ Duration quietTimeout,
+ int maxMessages,
+ Consumer... consumers)
+ throws ExecutionException, InterruptedException {
+ AtomicInteger messagesReceived = new AtomicInteger();
+ receiveMessages(
+ (consumer, message) -> {
+ messageHandler.accept(consumer, message);
+ return messagesReceived.incrementAndGet() < maxMessages;
+ }, quietTimeout, consumers);
+ }
+
+ /**
+ * Receive messages concurrently from multiple consumers and handles them using the provided message handler.
+ *
+ * @param messageHandler the message handler
+ * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages
+ * @param consumers the consumers to receive messages from
+ * @param the message value type
+ */
+ public static void receiveMessagesInThreads(BiFunction, Message, Boolean> messageHandler,
+ final Duration quietTimeout,
+ Consumer... consumers) {
+ receiveMessagesInThreads(messageHandler, quietTimeout, Arrays.stream(consumers).sequential());
+ }
+
+ /**
+ * Receive messages concurrently from multiple consumers and handles them using the provided message handler.
+ *
+ * @param messageHandler the message handler
+ * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages
+ * @param consumers the consumers to receive messages from
+ * @param the message value type
+ */
+ public static void receiveMessagesInThreads(BiFunction, Message, Boolean> messageHandler,
+ final Duration quietTimeout,
+ Stream> consumers) {
+ FutureUtil.waitForAll(consumers.map(consumer -> {
+ return CompletableFuture.runAsync(() -> {
+ try {
+ while (!Thread.currentThread().isInterrupted()) {
+ Message msg = consumer.receive((int) quietTimeout.toMillis(), TimeUnit.MILLISECONDS);
+ if (msg != null) {
+ if (!messageHandler.apply(consumer, msg)) {
+ break;
+ }
+ } else {
+ break;
+ }
+ }
+ } catch (PulsarClientException e) {
+ throw new CompletionException(e);
+ }
+ }, runnable -> {
+ Thread thread = new Thread(runnable, "Consumer-" + consumer.getConsumerName());
+ thread.start();
+ });
+ }).toList()).join();
+ }
+
+ private static long mockConsumerIdGenerator = 0;
+
+ public static org.apache.pulsar.broker.service.Consumer createMockConsumer(String consumerName) {
+ long consumerId = mockConsumerIdGenerator++;
+ return createMockConsumer(consumerName, consumerName + " consumerId:" + consumerId, consumerId);
+ }
+
+ public static org.apache.pulsar.broker.service.Consumer createMockConsumer(String consumerName, String toString, long consumerId) {
+ // without stubOnly, the mock will record method invocations and could run into OOME
+ org.apache.pulsar.broker.service.Consumer
+ consumer = mock(org.apache.pulsar.broker.service.Consumer.class, Mockito.withSettings().stubOnly());
+ when(consumer.consumerName()).thenReturn(consumerName);
+ when(consumer.toString()).thenReturn(consumerName + " consumerId:" + consumerId);
+ when(consumer.consumerId()).thenReturn(consumerId);
+ return consumer;
+ }
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtilTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtilTest.java
new file mode 100644
index 0000000000000..90b917a319c71
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtilTest.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiFunction;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class BrokerTestUtilTest {
+ @Test
+ public void testReceiveMessagesQuietTime() throws Exception {
+ // Mock consumers
+ Consumer consumer1 = mock(Consumer.class);
+ Consumer consumer2 = mock(Consumer.class);
+
+ long consumer1DelayMs = 300L;
+ long consumer2DelayMs = 400L;
+ long quietTimeMs = 500L;
+
+ // Define behavior for receiveAsync with delay
+ AtomicBoolean consumer1FutureContinueSupplying = new AtomicBoolean(true);
+ when(consumer1.receiveAsync()).thenAnswer(invocation -> {
+ if (consumer1FutureContinueSupplying.get()) {
+ CompletableFuture messageCompletableFuture =
+ CompletableFuture.supplyAsync(() -> mock(Message.class),
+ CompletableFuture.delayedExecutor(consumer1DelayMs, TimeUnit.MILLISECONDS));
+ consumer1FutureContinueSupplying.set(false);
+ // continue supplying while the future is cancelled or timed out
+ FutureUtil.whenCancelledOrTimedOut(messageCompletableFuture, () -> {
+ consumer1FutureContinueSupplying.set(true);
+ });
+ return messageCompletableFuture;
+ } else {
+ return new CompletableFuture<>();
+ }
+ });
+ AtomicBoolean consumer2FutureContinueSupplying = new AtomicBoolean(true);
+ when(consumer2.receiveAsync()).thenAnswer(invocation -> {
+ if (consumer2FutureContinueSupplying.get()) {
+ CompletableFuture messageCompletableFuture =
+ CompletableFuture.supplyAsync(() -> mock(Message.class),
+ CompletableFuture.delayedExecutor(consumer2DelayMs, TimeUnit.MILLISECONDS));
+ consumer2FutureContinueSupplying.set(false);
+ // continue supplying while the future is cancelled or timed out
+ FutureUtil.whenCancelledOrTimedOut(messageCompletableFuture, () -> {
+ consumer2FutureContinueSupplying.set(true);
+ });
+ return messageCompletableFuture;
+ } else {
+ return new CompletableFuture<>();
+ }
+ });
+
+ // Atomic variables to track message handling
+ AtomicInteger messageCount = new AtomicInteger(0);
+
+ // Message handler
+ BiFunction, Message, Boolean> messageHandler = (consumer, msg) -> {
+ messageCount.incrementAndGet();
+ return true;
+ };
+
+ // Track start time
+ long startTime = System.nanoTime();
+
+ // Call receiveMessages method
+ BrokerTestUtil.receiveMessages(messageHandler, Duration.ofMillis(quietTimeMs), consumer1, consumer2);
+
+ // Track end time
+ long endTime = System.nanoTime();
+
+ // Verify that messages were attempted to be received
+ verify(consumer1, times(3)).receiveAsync();
+ verify(consumer2, times(2)).receiveAsync();
+
+ // Verify that the message handler was called
+ assertEquals(messageCount.get(), 2);
+
+ // Verify the time spent is as expected (within a reasonable margin)
+ long durationMillis = TimeUnit.NANOSECONDS.toMillis(endTime - startTime);
+ assertThat(durationMillis).isBetween(consumer2DelayMs + quietTimeMs,
+ consumer2DelayMs + quietTimeMs + (quietTimeMs / 2));
+ }
+}
\ No newline at end of file
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java
index 78ae046b0c8c8..0d5fc4e19d5b3 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java
@@ -20,6 +20,8 @@
import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedDataAttributeName;
import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedRoleAttributeName;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
@@ -29,15 +31,22 @@
import static org.testng.Assert.assertTrue;
import com.google.common.collect.Sets;
import java.io.IOException;
+import java.util.LinkedHashSet;
import java.util.Set;
+import java.util.concurrent.CompletableFuture;
import javax.naming.AuthenticationException;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
+import lombok.Cleanup;
import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.authentication.AuthenticationDataCommand;
import org.apache.pulsar.broker.authentication.AuthenticationDataHttps;
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
import org.apache.pulsar.broker.authentication.AuthenticationProvider;
import org.apache.pulsar.broker.authentication.AuthenticationService;
+import org.apache.pulsar.broker.authentication.AuthenticationState;
+import org.apache.pulsar.broker.web.AuthenticationFilter;
+import org.apache.pulsar.common.api.AuthData;
import org.testng.Assert;
import org.testng.annotations.Test;
@@ -166,6 +175,123 @@ public void testAuthenticationHttpRequestResponseWithAnonymousRole() throws Exce
service.close();
}
+ @Test
+ public void testHttpRequestWithMultipleProviders() throws Exception {
+ ServiceConfiguration config = new ServiceConfiguration();
+ Set providersClassNames = new LinkedHashSet<>();
+ providersClassNames.add(MockAuthenticationProviderAlwaysFail.class.getName());
+ providersClassNames.add(MockHttpAuthenticationProvider.class.getName());
+ config.setAuthenticationProviders(providersClassNames);
+ config.setAuthenticationEnabled(true);
+ @Cleanup
+ AuthenticationService service = new AuthenticationService(config);
+
+ HttpServletRequest request = mock(HttpServletRequest.class);
+
+ when(request.getParameter("role")).thenReturn("success-role1");
+ assertTrue(service.authenticateHttpRequest(request, (HttpServletResponse) null));
+
+ when(request.getParameter("role")).thenReturn("");
+ assertThatThrownBy(() -> service.authenticateHttpRequest(request, (HttpServletResponse) null))
+ .isInstanceOf(AuthenticationException.class);
+
+ when(request.getParameter("role")).thenReturn("error-role1");
+ assertThatThrownBy(() -> service.authenticateHttpRequest(request, (HttpServletResponse) null))
+ .isInstanceOf(AuthenticationException.class);
+
+ when(request.getHeader(AuthenticationFilter.PULSAR_AUTH_METHOD_NAME)).thenReturn("http-auth");
+ assertThatThrownBy(() -> service.authenticateHttpRequest(request, (HttpServletResponse) null))
+ .isInstanceOf(RuntimeException.class);
+
+ HttpServletRequest requestForAuthenticationDataSource = mock(HttpServletRequest.class);
+ assertThatThrownBy(() -> service.authenticateHttpRequest(requestForAuthenticationDataSource,
+ (AuthenticationDataSource) null))
+ .isInstanceOf(AuthenticationException.class);
+
+ when(requestForAuthenticationDataSource.getParameter("role")).thenReturn("error-role2");
+ assertThatThrownBy(() -> service.authenticateHttpRequest(requestForAuthenticationDataSource,
+ (AuthenticationDataSource) null))
+ .isInstanceOf(AuthenticationException.class);
+
+ when(requestForAuthenticationDataSource.getParameter("role")).thenReturn("success-role2");
+ assertThat(service.authenticateHttpRequest(requestForAuthenticationDataSource,
+ (AuthenticationDataSource) null)).isEqualTo("role2");
+ }
+
+ public static class MockHttpAuthenticationProvider implements AuthenticationProvider {
+ @Override
+ public void close() throws IOException {
+ }
+
+ @Override
+ public void initialize(ServiceConfiguration config) throws IOException {
+ }
+
+ @Override
+ public String getAuthMethodName() {
+ return "http-auth";
+ }
+
+ private String getRole(HttpServletRequest request) {
+ String role = request.getParameter("role");
+ if (role != null) {
+ String[] s = role.split("-");
+ if (s.length == 2 && s[0].equals("success")) {
+ return s[1];
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletResponse response) {
+ String role = getRole(request);
+ if (role != null) {
+ return true;
+ }
+ throw new RuntimeException("test authentication failed");
+ }
+
+ @Override
+ public String authenticate(AuthenticationDataSource authData) throws AuthenticationException {
+ return authData.getCommandData();
+ }
+
+ @Override
+ public AuthenticationState newHttpAuthState(HttpServletRequest request) throws AuthenticationException {
+ String role = getRole(request);
+ if (role != null) {
+ return new AuthenticationState() {
+ @Override
+ public String getAuthRole() throws AuthenticationException {
+ return role;
+ }
+
+ @Override
+ public AuthData authenticate(AuthData authData) throws AuthenticationException {
+ return null;
+ }
+
+ @Override
+ public AuthenticationDataSource getAuthDataSource() {
+ return new AuthenticationDataCommand(role);
+ }
+
+ @Override
+ public boolean isComplete() {
+ return true;
+ }
+
+ @Override
+ public CompletableFuture authenticateAsync(AuthData authData) {
+ return AuthenticationState.super.authenticateAsync(authData);
+ }
+ };
+ }
+ throw new RuntimeException("new http auth failed");
+ }
+ }
+
public static class MockAuthenticationProvider implements AuthenticationProvider {
@Override
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java
index 68356b1140d99..ca0f414dd067d 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java
@@ -41,6 +41,7 @@
import javax.ws.rs.container.TimeoutHandler;
import lombok.AllArgsConstructor;
import lombok.Data;
+import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.authentication.AuthenticationProviderTls;
@@ -713,5 +714,9 @@ protected void reconnectAllConnections() throws Exception {
reconnectAllConnections((PulsarClientImpl) pulsarClient);
}
+ protected void logTopicStats(String topic) {
+ BrokerTestUtil.logTopicStats(log, admin, topic);
+ }
+
private static final Logger log = LoggerFactory.getLogger(MockedPulsarServiceBaseTest.class);
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerConfigTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerConfigTest.java
new file mode 100644
index 0000000000000..f9ce0d5019495
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerConfigTest.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.service;
+
+import static org.testng.Assert.assertEquals;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
+import org.apache.pulsar.broker.BrokerTestUtil;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+@Test(groups = "broker")
+public class ManagedLedgerConfigTest extends ProducerConsumerBase {
+
+ @BeforeClass(alwaysRun = true)
+ @Override
+ protected void setup() throws Exception {
+ super.internalSetup();
+ super.producerBaseSetup();
+ }
+
+ @AfterClass(alwaysRun = true)
+ @Override
+ protected void cleanup() throws Exception {
+ super.internalCleanup();
+ }
+
+ @DataProvider(name = "booleans")
+ public Object[][] booleans() {
+ return new Object[][] {
+ {true},
+ {false},
+ };
+ }
+
+ @Test(dataProvider = "booleans")
+ public void testConfigPersistIndividualAckAsLongArray(boolean enabled) throws Exception {
+ pulsar.getConfiguration().setManagedLedgerPersistIndividualAckAsLongArray(enabled);
+ final String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp");
+ admin.topics().createNonPartitionedTopic(tpName);
+ PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, true).get().get();
+ ManagedLedgerConfig mlConf = topic.getManagedLedger().getConfig();
+ assertEquals(mlConf.isPersistIndividualAckAsLongArray(), enabled);
+
+ // cleanup.
+ admin.topics().delete(tpName);
+ }
+}
+
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java
index 143557b008b23..609430db6df05 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java
@@ -26,6 +26,7 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.broker.ServiceConfiguration;
@@ -94,7 +95,7 @@ public void testTopicUnloadAfterSessionRebuild(boolean enableSystemTopic, Class
admin2.namespaces().unload(defaultNamespace);
// Confirm all brokers registered.
- Awaitility.await().untilAsserted(() -> {
+ Awaitility.await().atMost(Duration.ofSeconds(20)).untilAsserted(() -> {
assertEquals(getAvailableBrokers(pulsar1).size(), 2);
assertEquals(getAvailableBrokers(pulsar2).size(), 2);
});
@@ -160,7 +161,21 @@ public void testTopicUnloadAfterSessionRebuild(boolean enableSystemTopic, Class
// Verify: the topic on broker-2 is fine.
Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> {
CompletableFuture> future = pulsar1.getBrokerService().getTopic(topicName, false);
- assertTrue(future == null || future.isCompletedExceptionally());
+ log.info("broker 1 topics {}", pulsar1.getBrokerService().getTopics().keys());
+ log.info("broker 2 topics {}", pulsar2.getBrokerService().getTopics().keys());
+ log.info("broker 1 bundles {}", pulsar1.getNamespaceService().getOwnershipCache().getOwnedBundles()
+ .keySet().stream().map(k -> k.getNamespaceObject().toString() + "/" + k.getBundleRange())
+ .filter(s -> s.contains(defaultNamespace)).collect(Collectors.toList()));
+ log.info("broker 2 bundles {}", pulsar2.getNamespaceService().getOwnershipCache().getOwnedBundles()
+ .keySet().stream().map(k -> k.getNamespaceObject().toString() + "/" + k.getBundleRange())
+ .filter(s -> s.contains(defaultNamespace)).collect(Collectors.toList()));
+ log.info("future: {}, isDone: {}, isCompletedExceptionally: {}",
+ future, future == null ? "null" : future.isDone(),
+ future, future == null ? "null" : future.isCompletedExceptionally());
+ assertTrue(future == null
+ || !pulsar1.getBrokerService().getTopics().containsKey(topicName)
+ || (future.isDone() && !future.isCompletedExceptionally() && future.get().isEmpty())
+ || future.isCompletedExceptionally());
});
Topic broker2Topic3 = pulsar2.getBrokerService().getTopic(topicName, false).join().get();
assertNotNull(broker2Topic3);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java
index 086d434b81d2f..dc6c53154e607 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java
@@ -457,10 +457,10 @@ public void testEnableTopicDelayedDelivery() throws Exception {
break;
}
}
- producer.newMessage().value("long-tick-msg").deliverAfter(2, TimeUnit.SECONDS).send();
+ producer.newMessage().value("long-tick-msg").deliverAfter(3, TimeUnit.SECONDS).send();
msg = consumer.receive(1, TimeUnit.SECONDS);
assertNull(msg);
- msg = consumer.receive(3, TimeUnit.SECONDS);
+ msg = consumer.receive(4, TimeUnit.SECONDS);
assertNotNull(msg);
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java
index 5b750a0b9c2e5..243a5ccadb369 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java
@@ -423,10 +423,11 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex
@Cleanup
Producer producer = client.newProducer(Schema.STRING).topic(topic).enableBatching(false).create();
+ String subName = "test_sub";
@Cleanup
Consumer consumer = client.newConsumer(Schema.STRING)
.topic(topic)
- .subscriptionName("test_sub")
+ .subscriptionName(subName)
.subscriptionType(SubscriptionType.Shared)
.messageListener((MessageListener) (consumer1, msg) -> {
try {
@@ -452,7 +453,13 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex
Multimap metricsMap = parseMetrics(metricsStr);
Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes");
- Assert.assertTrue(metrics.size() > 0);
+ Collection subMetrics = metricsMap.get("pulsar_subscription_delayed_message_index_size_bytes");
+ assertFalse(metrics.isEmpty());
+ if (exposeTopicLevelMetrics) {
+ assertFalse(subMetrics.isEmpty());
+ } else {
+ assertTrue(subMetrics.isEmpty());
+ }
int topicLevelNum = 0;
int namespaceLevelNum = 0;
@@ -461,14 +468,20 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex
if (exposeTopicLevelMetrics && metric.tags.get("topic").equals(topic)) {
Assert.assertTrue(metric.value > 0);
topicLevelNum++;
- if ("test_sub".equals(metric.tags.get("subscription"))) {
- subscriptionLevelNum++;
- }
} else if (!exposeTopicLevelMetrics && metric.tags.get("namespace").equals(namespace)) {
Assert.assertTrue(metric.value > 0);
namespaceLevelNum++;
}
}
+ if (exposeTopicLevelMetrics) {
+ for (Metric metric : subMetrics) {
+ if (metric.tags.get("topic").equals(topic) &&
+ subName.equals(metric.tags.get("subscription"))) {
+ Assert.assertTrue(metric.value > 0);
+ subscriptionLevelNum++;
+ }
+ }
+ }
if (exposeTopicLevelMetrics) {
Assert.assertTrue(topicLevelNum > 0);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java
index 1360dd7c4442b..fc09892f7fbab 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java
@@ -26,7 +26,6 @@
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.TimeUnit;
import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;
@@ -46,7 +45,6 @@
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.api.transaction.Transaction;
import org.apache.pulsar.client.api.transaction.TxnID;
-import org.apache.pulsar.common.util.collections.BitSetRecyclable;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.awaitility.Awaitility;
import org.testng.Assert;
@@ -233,10 +231,7 @@ public void txnAckTestBatchAndSharedSubMemoryDeleteTest() throws Exception {
(LinkedMap>) field.get(pendingAckHandle);
assertTrue(individualAckOfTransaction.isEmpty());
managedCursor = (ManagedCursorImpl) testPersistentSubscription.getCursor();
- field = ManagedCursorImpl.class.getDeclaredField("batchDeletedIndexes");
- field.setAccessible(true);
- final ConcurrentSkipListMap batchDeletedIndexes =
- (ConcurrentSkipListMap) field.get(managedCursor);
+ final var batchDeletedIndexes = managedCursor.getBatchDeletedIndexes();
if (retryCnt == 0) {
//one message are not ack
Awaitility.await().until(() -> {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java
index 19d6cc85c9ff6..9bf761c6509cb 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java
@@ -69,6 +69,7 @@ public class MLPendingAckStoreTest extends TransactionTestBase {
@BeforeClass
@Override
protected void setup() throws Exception {
+ conf.setAcknowledgmentAtBatchIndexLevelEnabled(true);
setUpBase(1, 1, NAMESPACE1 + "/test", 0);
}
@@ -300,4 +301,4 @@ private LinkedHashSet calculatePendingAckIndexes(List positionList,
}
return indexes;
}
-}
\ No newline at end of file
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionBrokerCacheTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionBrokerCacheTest.java
new file mode 100644
index 0000000000000..8fcda7e6d263e
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionBrokerCacheTest.java
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.api;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName;
+import static org.assertj.core.api.SoftAssertions.assertSoftly;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.Cleanup;
+import lombok.SneakyThrows;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryMBeanImpl;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.StickyKeyConsumerSelector;
+import org.apache.pulsar.broker.service.Topic;
+import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers;
+import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
+import org.awaitility.Awaitility;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionBrokerCacheTest extends ProducerConsumerBase {
+ private static final Logger log = LoggerFactory.getLogger(KeySharedSubscriptionBrokerCacheTest.class);
+ private static final String SUBSCRIPTION_NAME = "key_shared";
+
+ @BeforeClass(alwaysRun = true)
+ @Override
+ protected void setup() throws Exception {
+ super.internalSetup();
+ super.producerBaseSetup();
+ }
+
+ @Override
+ protected void doInitConf() throws Exception {
+ super.doInitConf();
+ conf.setUnblockStuckSubscriptionEnabled(false);
+ conf.setSubscriptionKeySharedUseConsistentHashing(true);
+ conf.setManagedLedgerCacheSizeMB(100);
+
+ // configure to evict entries after 30 seconds so that we can test retrieval from cache
+ conf.setManagedLedgerCacheEvictionTimeThresholdMillis(30000);
+ conf.setManagedLedgerCacheEvictionIntervalMs(30000);
+
+ // Important: this is currently necessary to make use of cache for replay queue reads
+ conf.setCacheEvictionByMarkDeletedPosition(true);
+
+ conf.setManagedLedgerMaxReadsInFlightSizeInMB(100);
+ }
+
+ @AfterClass(alwaysRun = true)
+ @Override
+ protected void cleanup() throws Exception {
+ super.internalCleanup();
+ }
+
+ @AfterMethod(alwaysRun = true)
+ public void resetAfterMethod() throws Exception {
+ List list = admin.namespaces().getTopics("public/default");
+ for (String topicName : list){
+ if (!pulsar.getBrokerService().isSystemTopic(topicName)) {
+ admin.topics().delete(topicName, false);
+ }
+ }
+ pulsarTestContext.getMockBookKeeper().setReadHandleInterceptor(null);
+ }
+
+ // Use a fixed seed to make the tests using random values deterministic
+ // When a test fails, it's possible to re-run it to reproduce the issue
+ private static final Random random = new Random(1);
+
+ private Producer createProducer(String topic, boolean enableBatch) throws PulsarClientException {
+ Producer producer = null;
+ if (enableBatch) {
+ producer = pulsarClient.newProducer(Schema.INT32)
+ .topic(topic)
+ .enableBatching(true)
+ .maxPendingMessages(2001)
+ .batcherBuilder(BatcherBuilder.KEY_BASED)
+ .create();
+ } else {
+ producer = pulsarClient.newProducer(Schema.INT32)
+ .topic(topic)
+ .maxPendingMessages(2001)
+ .enableBatching(false)
+ .create();
+ }
+ return producer;
+ }
+
+ private StickyKeyConsumerSelector getSelector(String topic, String subscription) {
+ return getStickyKeyDispatcher(topic, subscription).getSelector();
+ }
+
+ @SneakyThrows
+ private PersistentStickyKeyDispatcherMultipleConsumers getStickyKeyDispatcher(String topic, String subscription) {
+ Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get();
+ PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription);
+ PersistentStickyKeyDispatcherMultipleConsumers dispatcher =
+ (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher();
+ return dispatcher;
+ }
+
+ @Test(invocationCount = 1)
+ public void testReplayQueueReadsGettingCached() throws Exception {
+ String topic = newUniqueName("testReplayQueueReadsGettingCached");
+ int numberOfKeys = 100;
+ long pauseTime = 100L;
+ long testStartNanos = System.nanoTime();
+
+ @Cleanup
+ Producer producer = createProducer(topic, false);
+
+ // create a consumer and close it to create a subscription
+ pulsarClient.newConsumer(Schema.INT32)
+ .topic(topic)
+ .subscriptionName(SUBSCRIPTION_NAME)
+ .subscriptionType(SubscriptionType.Key_Shared)
+ .subscribe()
+ .close();
+
+ Set remainingMessageValues = Collections.synchronizedSet(new HashSet<>());
+ BlockingQueue, Message>> unackedMessages = new LinkedBlockingQueue<>();
+ AtomicBoolean c2MessagesShouldBeUnacked = new AtomicBoolean(true);
+ Set keysForC2 = new HashSet<>();
+ AtomicLong lastMessageTimestamp = new AtomicLong(System.currentTimeMillis());
+
+ MessageListener messageHandler = (consumer, msg) -> {
+ lastMessageTimestamp.set(System.currentTimeMillis());
+ synchronized (this) {
+ String key = msg.getKey();
+ if (c2MessagesShouldBeUnacked.get() && keysForC2.contains(key)) {
+ unackedMessages.add(Pair.of(consumer, msg));
+ return;
+ }
+ remainingMessageValues.remove(msg.getValue());
+ consumer.acknowledgeAsync(msg);
+ }
+ };
+
+ pulsarTestContext.getMockBookKeeper().setReadHandleInterceptor((ledgerId, firstEntry, lastEntry, entries) -> {
+ log.error("Attempting to read from BK when cache should be used. {}:{} to {}:{}", ledgerId, firstEntry,
+ ledgerId, lastEntry);
+ return CompletableFuture.failedFuture(
+ new ManagedLedgerException.NonRecoverableLedgerException(
+ "Should not read from BK since cache should be used."));
+ });
+
+ // Adding a new consumer.
+ @Cleanup
+ Consumer c1 = pulsarClient.newConsumer(Schema.INT32)
+ .topic(topic)
+ .consumerName("c1")
+ .subscriptionName(SUBSCRIPTION_NAME)
+ .subscriptionType(SubscriptionType.Key_Shared)
+ .messageListener(messageHandler)
+ .subscribe();
+
+ @Cleanup
+ Consumer c2 = pulsarClient.newConsumer(Schema.INT32)
+ .topic(topic)
+ .consumerName("c2")
+ .subscriptionName(SUBSCRIPTION_NAME)
+ .subscriptionType(SubscriptionType.Key_Shared)
+ .messageListener(messageHandler)
+ .subscribe();
+
+ @Cleanup
+ Consumer c3 = pulsarClient.newConsumer(Schema.INT32)
+ .topic(topic)
+ .consumerName("c3")
+ .subscriptionName(SUBSCRIPTION_NAME)
+ .subscriptionType(SubscriptionType.Key_Shared)
+ .messageListener(messageHandler)
+ .subscribe();
+
+ PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getStickyKeyDispatcher(topic, SUBSCRIPTION_NAME);
+ StickyKeyConsumerSelector selector = dispatcher.getSelector();
+
+ // find keys that will be assigned to c2
+ for (int i = 0; i < numberOfKeys; i++) {
+ String key = String.valueOf(i);
+ byte[] keyBytes = key.getBytes(UTF_8);
+ int hash = StickyKeyConsumerSelector.makeStickyKeyHash(keyBytes);
+ if (selector.select(hash).consumerName().equals("c2")) {
+ keysForC2.add(key);
+ }
+ }
+
+ // close c2
+ c2.close();
+
+ // produce messages with random keys
+ for (int i = 0; i < 1000; i++) {
+ String key = String.valueOf(random.nextInt(numberOfKeys));
+ //log.info("Producing message with key: {} value: {}", key, i);
+ remainingMessageValues.add(i);
+ producer.newMessage()
+ .key(key)
+ .value(i)
+ .send();
+ }
+
+ // reconnect c2
+ c2 = pulsarClient.newConsumer(Schema.INT32)
+ .topic(topic)
+ .consumerName("c2")
+ .subscriptionName(SUBSCRIPTION_NAME)
+ .subscriptionType(SubscriptionType.Key_Shared)
+ .messageListener(messageHandler)
+ .startPaused(true)
+ .subscribe();
+
+ // ack the unacked messages to unblock c2 keys
+ c2MessagesShouldBeUnacked.set(false);
+ Pair, Message> consumerMessagePair;
+ while ((consumerMessagePair = unackedMessages.poll()) != null) {
+ messageHandler.received(consumerMessagePair.getLeft(), consumerMessagePair.getRight());
+ }
+
+ // produce more messages with random keys
+ for (int i = 0; i < 1000; i++) {
+ String key = String.valueOf(random.nextInt(numberOfKeys));
+ //log.info("Producing message with key: {} value: {}", key, i);
+ remainingMessageValues.add(i);
+ producer.newMessage()
+ .key(key)
+ .value(i)
+ .send();
+ }
+
+ c2.resume();
+
+ Awaitility.await().atMost(Duration.ofSeconds(10)).until(() -> {
+ return remainingMessageValues.isEmpty()
+ || System.currentTimeMillis() - lastMessageTimestamp.get() > 50 * pauseTime;
+ });
+
+ try {
+ assertSoftly(softly -> {
+ softly.assertThat(remainingMessageValues).as("remainingMessageValues").isEmpty();
+ ManagedLedgerFactoryMBeanImpl cacheStats =
+ ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getMbean();
+ cacheStats.refreshStats(System.nanoTime() - testStartNanos, TimeUnit.NANOSECONDS);
+ softly.assertThat(cacheStats.getCacheHitsRate()).as("cache hits").isGreaterThan(0.0);
+ softly.assertThat(cacheStats.getCacheMissesRate()).as("cache misses").isEqualTo(0.0);
+ softly.assertThat(cacheStats.getNumberOfCacheEvictions()).as("cache evictions").isEqualTo(0);
+ });
+ } finally {
+ logTopicStats(topic);
+ }
+ }
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java
index 91b97fa475817..d0e72deb87fc2 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java
@@ -464,7 +464,7 @@ public void testRetryTopicNameForCompatibility () throws Exception {
}
/**
- * The test is disabled {@link https://github.com/apache/pulsar/issues/2647}.
+ * Test retry topic with multiple topics
* @throws Exception
*/
@Test
@@ -482,7 +482,6 @@ public void testRetryTopicWithMultiTopic() throws Exception {
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.enableRetry(true)
- .ackTimeout(1, TimeUnit.SECONDS)
.deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build())
.receiverQueueSize(100)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
@@ -518,6 +517,7 @@ public void testRetryTopicWithMultiTopic() throws Exception {
Message message = consumer.receive();
log.info("consumer received message : {} {} - total = {}",
message.getMessageId(), new String(message.getData()), ++totalReceived);
+ consumer.reconsumeLater(message, 1, TimeUnit.SECONDS);
} while (totalReceived < sendMessages * (maxRedeliveryCount + 1));
int totalInDeadLetter = 0;
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java
index a83283bc267b5..6d9025fd966b3 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java
@@ -116,6 +116,40 @@ public void testAckResponse() throws PulsarClientException, InterruptedException
Assert.assertTrue(e.getCause() instanceof PulsarClientException.NotAllowedException);
}
}
+ @Test(timeOut = 30000)
+ public void testAckReceipt() throws Exception {
+ String topic = "testAckReceipt";
+ @Cleanup
+ Producer producer = pulsarClient.newProducer(Schema.INT32)
+ .topic(topic)
+ .enableBatching(false)
+ .create();
+ @Cleanup
+ ConsumerImpl consumer = (ConsumerImpl) pulsarClient.newConsumer(Schema.INT32)
+ .topic(topic)
+ .subscriptionName("sub")
+ .isAckReceiptEnabled(true)
+ .subscribe();
+ for (int i = 0; i < 10; i++) {
+ producer.send(i);
+ }
+ Message message = consumer.receive();
+ MessageId messageId = message.getMessageId();
+ consumer.acknowledgeCumulativeAsync(messageId).get();
+ consumer.acknowledgeCumulativeAsync(messageId).get();
+ consumer.close();
+ @Cleanup
+ ConsumerImpl consumer2 = (ConsumerImpl) pulsarClient.newConsumer(Schema.INT32)
+ .topic(topic)
+ .subscriptionName("sub")
+ .isAckReceiptEnabled(true)
+ .acknowledgmentGroupTime(0, TimeUnit.SECONDS)
+ .subscribe();
+ message = consumer2.receive();
+ messageId = message.getMessageId();
+ consumer2.acknowledgeCumulativeAsync(messageId).get();
+ consumer2.acknowledgeCumulativeAsync(messageId).get();
+ }
@Test
public void testIndividualAck() throws Exception {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerCloseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerCloseTest.java
index 1141af88e72b0..623d8e7505e0d 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerCloseTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerCloseTest.java
@@ -18,6 +18,7 @@
*/
package org.apache.pulsar.client.impl;
+import java.time.Duration;
import lombok.Cleanup;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.pulsar.broker.service.Topic;
@@ -71,6 +72,46 @@ public Object[][] produceConf() {
};
}
+ /**
+ * Param1: Producer enableBatch or not
+ * Param2: Send in async way or not
+ */
+ @DataProvider(name = "brokenPipeline")
+ public Object[][] brokenPipeline() {
+ return new Object[][]{
+ {true},
+ {false}
+ };
+ }
+
+ @Test(dataProvider = "brokenPipeline")
+ public void testProducerCloseCallback2(boolean brokenPipeline) throws Exception {
+ initClient();
+ @Cleanup
+ ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer()
+ .topic("testProducerClose")
+ .sendTimeout(5, TimeUnit.SECONDS)
+ .maxPendingMessages(0)
+ .enableBatching(false)
+ .create();
+ final TypedMessageBuilder