From ecfa45f8fbb6394462f01d02836b897b9db94d0d Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 14 Jan 2025 16:30:44 +0800 Subject: [PATCH 01/31] [fix][broker] Continue using the next provider for http authentication if one fails (#23842) Signed-off-by: Zixuan Liu (cherry picked from commit f1f65a52eca65fd75171824c32be6b7597f0bc4c) Signed-off-by: Zixuan Liu (cherry picked from commit 81811771cb0dbad1658530030af749e62ffa06a6) --- .../authentication/AuthenticationService.java | 22 +-- .../auth/AuthenticationServiceTest.java | 126 ++++++++++++++++++ 2 files changed, 133 insertions(+), 15 deletions(-) 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/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 From 6f7c67ec844deb6c33d11c151ebd6cf97df358ca Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 13 Jan 2025 10:39:02 +0800 Subject: [PATCH 02/31] [fix][broker] Remove blocking calls from internalGetPartitionedStats (#23832) Signed-off-by: Zixuan Liu (cherry picked from commit dc6397093eb645fec1bfd6f127f78ad36be0a360) (cherry picked from commit 85a0f79bbf3cadeb9a3ebf58fdb93464e5eefa1d) --- .../admin/impl/PersistentTopicsBase.java | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) 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 -> { From 1bb0e945539bfa5f69a784b07b949f863a40818f Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 14 Jan 2025 22:39:14 +0800 Subject: [PATCH 03/31] [fix][misc] Honor dynamic log levels in log4j2.yaml (#23847) Signed-off-by: Zixuan Liu (cherry picked from commit 492a869b8dbdb65f394454754260548020ceac7e) (cherry picked from commit 72cc5bb192215c7afd197074efe68b73eb844aeb) --- bin/pulsar | 12 ++++++++---- bin/pulsar-admin-common.cmd | 12 ++++++++---- bin/pulsar-perf | 12 ++++++++---- docker/pulsar/Dockerfile | 2 -- 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/bin/pulsar b/bin/pulsar index 20ed1f7f22b0f..1165681237ae3 100755 --- a/bin/pulsar +++ b/bin/pulsar @@ -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..013d9ef4a55b6 100644 --- a/bin/pulsar-admin-common.cmd +++ b/bin/pulsar-admin-common.cmd @@ -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-perf b/bin/pulsar-perf index bdc1dc1ed8b8c..3aa3ada193663 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -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/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 From 8062add178af081f369ac9fabe76d72cbc9e2faf Mon Sep 17 00:00:00 2001 From: feynmanlin <315157973@qq.com> Date: Tue, 14 Jan 2025 10:41:59 +0800 Subject: [PATCH 04/31] [fix] [broker] Fix acknowledgeCumulativeAsync block when ackReceipt is enabled (#23841) (cherry picked from commit 999db2c7f5a8a3c09e2055fb47f9e3cb65e7d1d2) (cherry picked from commit c320fc9de2fcf6d7a4f48513e9e0ac3ff05a5ebf) --- .../pulsar/client/impl/ConsumerAckTest.java | 34 +++++++++++++++++++ ...sistentAcknowledgmentsGroupingTracker.java | 7 +++- 2 files changed, 40 insertions(+), 1 deletion(-) 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-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index c0ee13b346a0b..d30c3de0fd720 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -312,7 +312,12 @@ private CompletableFuture doCumulativeAck(MessageIdAdv messageId, Map readLock = acquireReadLock(); try { doCumulativeAckAsync(messageId, bitSet); - return readLock.map(__ -> currentCumulativeAckFuture).orElse(CompletableFuture.completedFuture(null)); + return readLock.map(__ -> { + if (consumer.isAckReceiptEnabled() && lastCumulativeAck.compareTo(messageId) == 0) { + return CompletableFuture.completedFuture(null); + } + return currentCumulativeAckFuture; + }).orElse(CompletableFuture.completedFuture(null)); } finally { readLock.ifPresent(Lock::unlock); } From dc8ad20ab9ed579be3529c9cdc0df747ade1780c Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 14 Jan 2025 16:58:11 +0800 Subject: [PATCH 05/31] [fix][broker] Fix possible mark delete NPE when batch index ack is enabled (#23833) (cherry picked from commit c92930f544119d80e3451a5027694d56a2662ff2) (cherry picked from commit 367546248b2f6fc078e0e7fb0fc5af6931f88662) --- .../mledger/impl/ManagedCursorImpl.java | 135 ++++++++---------- .../PendingAckInMemoryDeleteTest.java | 7 +- .../impl/MLPendingAckStoreTest.java | 3 +- .../sql/presto/util/CacheSizeAllocator.java | 4 +- 4 files changed, 64 insertions(+), 85 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 45219ca091fe0..c10ae2fbad343 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; @@ -60,6 +61,8 @@ import java.util.function.Function; import java.util.function.Predicate; 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; @@ -97,7 +100,6 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.StringProperty; 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; @@ -199,7 +201,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; @@ -657,6 +661,7 @@ private void recoverIndividualDeletedMessages(List i private void recoverBatchDeletedIndexes ( List batchDeletedIndexInfoList) { + Objects.requireNonNull(batchDeletedIndexes); lock.writeLock().lock(); try { this.batchDeletedIndexes.clear(); @@ -667,7 +672,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 +1332,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 +1967,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 +2013,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 +2142,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 +2280,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,11 +2289,8 @@ 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. @@ -2318,12 +2303,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 +2315,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 +3060,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 +3068,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()); @@ -3487,11 +3468,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 +3575,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 +3701,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/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-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/CacheSizeAllocator.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/CacheSizeAllocator.java index 387076cc6d7f7..af8203c58fffb 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/CacheSizeAllocator.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/CacheSizeAllocator.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.sql.presto.util; +import javax.annotation.Nullable; + /** * Cache size allocator. */ @@ -28,7 +30,7 @@ public interface CacheSizeAllocator { * * @return available cache size */ - long getAvailableCacheSize(); + @Nullable long[] getAckSet(); /** * Cost available cache. From 1a1e20e841c26e0e9e7aa64e8a5f5d407a44f9b8 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Tue, 14 Jan 2025 17:40:15 +0800 Subject: [PATCH 06/31] [fix][broker] PIP-399: Fix Metric Name for Delayed Queue (#23712) (cherry picked from commit 5be922b3fd47fb01006bd14f7396321470ef40f2) (cherry picked from commit c5c2f1f446b9a93fd2b720d095ac1a9a14f6cfc4) --- .../broker/stats/prometheus/TopicStats.java | 2 +- .../persistent/PersistentTopicTest.java | 23 +++++++++++++++---- 2 files changed, 19 insertions(+), 6 deletions(-) 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/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); From 6a29b160a23a24be541cdbbe9337fb51f0fb7cb9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Jan 2025 04:38:22 -0800 Subject: [PATCH 07/31] [improve] Support overriding java.net.preferIPv4Stack with OPTS (#23846) (cherry picked from commit aff9ec872b3f23aef90015dd9b81a8dccd6e9a07) (cherry picked from commit ad20850a486f7033c04bc35b2f3a750740c0e5ef) --- bin/bookkeeper | 2 +- bin/function-localrunner | 2 +- bin/pulsar | 2 +- bin/pulsar-admin-common.cmd | 2 +- bin/pulsar-admin-common.sh | 2 +- bin/pulsar-perf | 2 +- src/pulsar-io-gen.sh | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) 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 1165681237ae3..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" diff --git a/bin/pulsar-admin-common.cmd b/bin/pulsar-admin-common.cmd index 013d9ef4a55b6..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 ( 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 3aa3ada193663..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 diff --git a/src/pulsar-io-gen.sh b/src/pulsar-io-gen.sh index a3083e8f2b549..9f5b1a6058c7b 100755 --- a/src/pulsar-io-gen.sh +++ b/src/pulsar-io-gen.sh @@ -108,7 +108,7 @@ fi PULSAR_CLASSPATH="$PULSAR_JAR:$PULSAR_HOME/pulsar-io/docs/target/pulsar-io-docs.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`" OPTS="-cp $PULSAR_CLASSPATH $OPTS" OPTS="$OPTS $PULSAR_EXTRA_OPTS" From b5cce9279381ecfa14cf5bafd5519c7a79144df7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 16 Jan 2025 16:40:31 +0800 Subject: [PATCH 08/31] [fix][test]Fix flaky test testTopicUnloadAfterSessionRebuild (#23852) (cherry picked from commit b3641f068a862a738aacad14b0534cf012e4e584) (cherry picked from commit 816f867d0d27fee2ec29e91e620354b576bafda4) --- .../broker/service/ZkSessionExpireTest.java | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) 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); From 4b1e77b50138b4b9efe0b3adf5ef23e0a661c2a2 Mon Sep 17 00:00:00 2001 From: Clay Johnson Date: Thu, 16 Jan 2025 05:41:11 -0600 Subject: [PATCH 09/31] [improve][ci] Publish build scans to develocity.apache.org (#23851) (cherry picked from commit b55cd3e754e3bf7d8975ba3a3663eb9e64e139d6) (cherry picked from commit 28e005508bf69b6cee057cfd57d2d4b1e76678cd) --- .github/workflows/ci-maven-cache-update.yaml | 2 +- .../workflows/ci-owasp-dependency-check.yaml | 2 +- .github/workflows/pulsar-ci-flaky.yaml | 2 +- .github/workflows/pulsar-ci.yaml | 18 +++++++++--------- .mvn/develocity.xml | 10 +++++----- .mvn/extensions.xml | 4 ++-- 6 files changed, 19 insertions(+), 19 deletions(-) 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..6e60065b07ba9 100644 --- a/.github/workflows/ci-owasp-dependency-check.yaml +++ b/.github/workflows/ci-owasp-dependency-check.yaml @@ -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..7df18f59b880e 100644 --- a/.github/workflows/pulsar-ci-flaky.yaml +++ b/.github/workflows/pulsar-ci-flaky.yaml @@ -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..42d840028cab7 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -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 From 8a910b8ac9940f8739e79ce588ac42844c8642d4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 16 Jan 2025 22:08:44 -0800 Subject: [PATCH 10/31] [fix][broker] Revert "[fix][broker] Cancel possible pending replay read in cancelPendingRead (#23384)" (#23855) (cherry picked from commit ea56ada4f3985c93b93c64d1361b3111cd98a37f) (cherry picked from commit 7387653d48cf70cc9d8eab97294e5dec25f768da) --- .../broker/service/AbstractDispatcherMultipleConsumers.java | 4 ++++ .../persistent/PersistentDispatcherMultipleConsumers.java | 3 +-- 2 files changed, 5 insertions(+), 2 deletions(-) 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/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; } } From 4a12bafe79ada2a8d815172e388a34fa14a66442 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 17 Jan 2025 14:48:46 +0800 Subject: [PATCH 11/31] [fix][client] Orphan producer when concurrently calling producer closing and reconnection (#23853) (cherry picked from commit 56adefaf3a244514a7eacbee17ac31b1a8739391) (cherry picked from commit aea49001dedb0760bb762ee09de1a335fa86c12a) --- .../pulsar/client/impl/ProducerCloseTest.java | 41 ++++++ .../client/impl/ProducerReconnectionTest.java | 130 ++++++++++++++++++ .../pulsar/client/impl/ConnectionHandler.java | 7 +- .../pulsar/client/impl/ProducerImpl.java | 30 ++-- .../pulsar/client/impl/ProducerImplTest.java | 36 ----- 5 files changed, 196 insertions(+), 48 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerReconnectionTest.java 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 messageBuilder = producer.newMessage(); + final TypedMessageBuilder value = messageBuilder.value("test-msg".getBytes(StandardCharsets.UTF_8)); + producer.getClientCnx().channel().config().setAutoRead(false); + final CompletableFuture completableFuture = value.sendAsync(); + producer.closeAsync(); + Thread.sleep(3000); + if (brokenPipeline) { + //producer.getClientCnx().channel().config().setAutoRead(true); + producer.getClientCnx().channel().close(); + } else { + producer.getClientCnx().channel().config().setAutoRead(true); + } + Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> { + System.out.println(1); + Assert.assertTrue(completableFuture.isDone()); + }); + } + @Test(timeOut = 10_000) public void testProducerCloseCallback() throws Exception { initClient(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerReconnectionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerReconnectionTest.java new file mode 100644 index 0000000000000..e05e666d62efd --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerReconnectionTest.java @@ -0,0 +1,130 @@ +/* + * 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.impl; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertTrue; +import java.nio.charset.StandardCharsets; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.ServerCnx; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class ProducerReconnectionTest 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(); + } + + @Test + public void testConcurrencyReconnectAndClose() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp_"); + admin.topics().createNonPartitionedTopic(topicName); + PulsarClientImpl client = (PulsarClientImpl) pulsarClient; + + // Create producer which will run with special steps. + ProducerBuilderImpl producerBuilder = (ProducerBuilderImpl) client.newProducer() + .blockIfQueueFull(false).maxPendingMessages(1).producerName("p1") + .enableBatching(true).topic(topicName); + CompletableFuture> producerFuture = new CompletableFuture<>(); + AtomicBoolean reconnectionStartTrigger = new AtomicBoolean(); + CountDownLatch reconnectingSignal = new CountDownLatch(1); + CountDownLatch closedSignal = new CountDownLatch(1); + ProducerImpl producer = new ProducerImpl<>(client, topicName, producerBuilder.getConf(), producerFuture, + -1, Schema.BYTES, null, Optional.empty()) { + @Override + ConnectionHandler initConnectionHandler() { + ConnectionHandler connectionHandler = super.initConnectionHandler(); + ConnectionHandler spyConnectionHandler = spy(connectionHandler); + doAnswer(invocation -> { + boolean result = (boolean) invocation.callRealMethod(); + if (reconnectionStartTrigger.get()) { + log.info("[testConcurrencyReconnectAndClose] verified state for reconnection"); + reconnectingSignal.countDown(); + closedSignal.await(); + log.info("[testConcurrencyReconnectAndClose] reconnected"); + } + return result; + }).when(spyConnectionHandler).isValidStateForReconnection(); + return spyConnectionHandler; + } + }; + log.info("[testConcurrencyReconnectAndClose] producer created"); + producerFuture.get(5, TimeUnit.SECONDS); + + // Reconnect. + log.info("[testConcurrencyReconnectAndClose] trigger a reconnection"); + ServerCnx serverCnx = (ServerCnx) pulsar.getBrokerService().getTopic(topicName, false).join() + .get().getProducers().values().iterator().next().getCnx(); + reconnectionStartTrigger.set(true); + serverCnx.ctx().close(); + producer.sendAsync("1".getBytes(StandardCharsets.UTF_8)); + Awaitility.await().untilAsserted(() -> { + assertNotEquals(producer.getPendingQueueSize(), 0); + }); + + // Close producer when reconnecting. + reconnectingSignal.await(); + log.info("[testConcurrencyReconnectAndClose] producer close"); + producer.closeAsync(); + Awaitility.await().untilAsserted(() -> { + HandlerState.State state1 = producer.getState(); + assertTrue(state1 == HandlerState.State.Closed || state1 == HandlerState.State.Closing); + }); + // give another thread time to call "signalToChangeStateToConnecting.await()". + closedSignal.countDown(); + + // Wait for reconnection. + Thread.sleep(3000); + + HandlerState.State state2 = producer.getState(); + log.info("producer state: {}", state2); + assertTrue(state2 == HandlerState.State.Closed || state2 == HandlerState.State.Closing); + assertEquals(producer.getPendingQueueSize(), 0); + + // Verify: ref is expected. + producer.close(); + admin.topics().delete(topicName); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java index 2b7fb90b14a47..8ec8a47dfbabd 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl; +import com.google.common.annotations.VisibleForTesting; import java.net.InetSocketAddress; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -168,13 +169,12 @@ public void connectionClosed(ClientCnx cnx) { duringConnect.set(false); state.client.getCnxPool().releaseConnection(cnx); if (CLIENT_CNX_UPDATER.compareAndSet(this, cnx, null)) { - if (!isValidStateForReconnection()) { + if (!state.changeToConnecting()) { log.info("[{}] [{}] Ignoring reconnection request (state: {})", state.topic, state.getHandlerName(), state.getState()); return; } long delayMs = backoff.next(); - state.setState(State.Connecting); log.info("[{}] [{}] Closed connection {} -- Will try again in {} s", state.topic, state.getHandlerName(), cnx.channel(), delayMs / 1000.0); @@ -208,7 +208,8 @@ protected long switchClientCnx(ClientCnx clientCnx) { return EPOCH_UPDATER.incrementAndGet(this); } - private boolean isValidStateForReconnection() { + @VisibleForTesting + public boolean isValidStateForReconnection() { State state = this.state.getState(); switch (state) { case Uninitialized: diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 92f8332dfb2c1..50209c009c75d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -268,15 +268,19 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration metadata = Collections.unmodifiableMap(new HashMap<>(conf.getProperties())); } - this.connectionHandler = new ConnectionHandler(this, + this.connectionHandler = initConnectionHandler(); + setChunkMaxMessageSize(); + grabCnx(); + } + + ConnectionHandler initConnectionHandler() { + return new ConnectionHandler(this, new BackoffBuilder() .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), TimeUnit.NANOSECONDS) .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) .setMandatoryStop(Math.max(100, conf.getSendTimeoutMs() - 100), TimeUnit.MILLISECONDS) .create(), - this); - setChunkMaxMessageSize(); - grabCnx(); + this); } private void setChunkMaxMessageSize() { @@ -1097,7 +1101,7 @@ public CompletableFuture handleOnce() { @Override - public CompletableFuture closeAsync() { + public synchronized CompletableFuture closeAsync() { final State currentState = getAndUpdateState(state -> { if (state == State.Closed) { return state; @@ -1124,11 +1128,11 @@ public CompletableFuture closeAsync() { CompletableFuture closeFuture = new CompletableFuture<>(); cnx.sendRequestWithId(cmd, requestId).handle((v, exception) -> { cnx.removeProducer(producerId); - closeAndClearPendingMessages(); if (exception == null || !cnx.ctx().channel().isActive()) { // Either we've received the success response for the close producer command from the broker, or the // connection did break in the meantime. In any case, the producer is gone. log.info("[{}] [{}] Closed Producer", topic, producerName); + closeAndClearPendingMessages(); closeFuture.complete(null); } else { closeFuture.completeExceptionally(exception); @@ -1714,6 +1718,12 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { // Because the state could have been updated while retrieving the connection, we set it back to connecting, // as long as the change from current state to connecting is a valid state change. if (!changeToConnecting()) { + if (getState() == State.Closing || getState() == State.Closed) { + // Producer was closed while reconnecting, close the connection to make sure the broker + // drops the producer on its side + failPendingMessages(cnx, + new PulsarClientException.ProducerFencedException("producer has been closed")); + } return CompletableFuture.completedFuture(null); } // We set the cnx reference before registering the producer on the cnx, so if the cnx breaks before creating @@ -1774,6 +1784,8 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { // Producer was closed while reconnecting, close the connection to make sure the broker // drops the producer on its side cnx.removeProducer(producerId); + failPendingMessages(cnx, + new PulsarClientException.ProducerFencedException("producer has been closed")); cnx.channel().close(); future.complete(null); return; @@ -1942,7 +1954,7 @@ private void closeProducerTasks() { private void resendMessages(ClientCnx cnx, long expectedEpoch) { cnx.ctx().channel().eventLoop().execute(() -> { - synchronized (this) { + synchronized (ProducerImpl.this) { if (getState() == State.Closing || getState() == State.Closed) { // Producer was closed while reconnecting, close the connection to make sure the broker // drops the producer on its side @@ -2098,7 +2110,7 @@ public void run(Timeout timeout) throws Exception { * This fails and clears the pending messages with the given exception. This method should be called from within the * ProducerImpl object mutex. */ - private void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { + private synchronized void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { if (cnx == null) { final AtomicInteger releaseCount = new AtomicInteger(); final boolean batchMessagingEnabled = isBatchMessagingEnabled(); @@ -2250,7 +2262,7 @@ private void batchMessageAndSend(boolean shouldScheduleNextBatchFlush) { } } - protected void processOpSendMsg(OpSendMsg op) { + protected synchronized void processOpSendMsg(OpSendMsg op) { if (op == null) { return; } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java index 4ead2839c51b9..7c6bcf2cf06be 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java @@ -25,14 +25,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; -import io.netty.util.HashedWheelTimer; import java.nio.ByteBuffer; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.conf.ClientConfigurationData; -import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.mockito.Mockito; import org.testng.annotations.Test; @@ -71,34 +65,4 @@ public void testPopulateMessageSchema() { assertTrue(producer.populateMessageSchema(msg, null)); verify(msg).setSchemaState(MessageImpl.SchemaState.Ready); } - - @Test - public void testClearPendingMessageWhenCloseAsync() { - PulsarClientImpl client = mock(PulsarClientImpl.class); - Mockito.doReturn(1L).when(client).newProducerId(); - ClientConfigurationData clientConf = new ClientConfigurationData(); - clientConf.setStatsIntervalSeconds(-1); - Mockito.doReturn(clientConf).when(client).getConfiguration(); - ConnectionPool connectionPool = mock(ConnectionPool.class); - Mockito.doReturn(1).when(connectionPool).genRandomKeyToSelectCon(); - Mockito.doReturn(connectionPool).when(client).getCnxPool(); - HashedWheelTimer timer = mock(HashedWheelTimer.class); - Mockito.doReturn(null).when(timer).newTimeout(Mockito.any(), Mockito.anyLong(), Mockito.any()); - Mockito.doReturn(timer).when(client).timer(); - ProducerConfigurationData producerConf = new ProducerConfigurationData(); - producerConf.setSendTimeoutMs(-1); - ProducerImpl producer = Mockito.spy(new ProducerImpl<>(client, "topicName", producerConf, null, 0, null, null, Optional.empty())); - - // make sure throw exception when send request to broker - ClientCnx clientCnx = mock(ClientCnx.class); - CompletableFuture tCompletableFuture = new CompletableFuture<>(); - tCompletableFuture.completeExceptionally(new PulsarClientException("error")); - when(clientCnx.sendRequestWithId(Mockito.any(), Mockito.anyLong())).thenReturn(tCompletableFuture); - Mockito.doReturn(clientCnx).when(producer).cnx(); - - // run closeAsync and verify - CompletableFuture voidCompletableFuture = producer.closeAsync(); - verify(producer).closeAndClearPendingMessages(); - } - } From 3d728d3b6a956a0d336350bcc576cc909c3ba8eb Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 17 Jan 2025 15:57:24 +0800 Subject: [PATCH 12/31] [fix][test] Add reconsumeLater call in RetryTopicTest#testRetryTopicWithMultiTopic. (#23857) (cherry picked from commit 80157955b69d97fa0568fad4127e34e30e9a2b85) (cherry picked from commit 7c5d7a889485776acaa4b92198894d65e8f3d6e7) --- .../java/org/apache/pulsar/client/api/RetryTopicTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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; From eea9df4f4c3033ea49257e58550c6cce4a5902ed Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 17 Jan 2025 13:02:42 +0200 Subject: [PATCH 13/31] [revert][sql][branch-3.0] Revert invalid cherry-pick merge conflict resolution result in commit 3675462, #23833 got resolved in a surprising way (cherry picked from commit e6cd85cad24dad98f42a8c69dc245ba819607db0) --- .../org/apache/pulsar/sql/presto/util/CacheSizeAllocator.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/CacheSizeAllocator.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/CacheSizeAllocator.java index af8203c58fffb..387076cc6d7f7 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/CacheSizeAllocator.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/CacheSizeAllocator.java @@ -18,8 +18,6 @@ */ package org.apache.pulsar.sql.presto.util; -import javax.annotation.Nullable; - /** * Cache size allocator. */ @@ -30,7 +28,7 @@ public interface CacheSizeAllocator { * * @return available cache size */ - @Nullable long[] getAckSet(); + long getAvailableCacheSize(); /** * Cost available cache. From a06f3367871844acb31c84b5ff097b58c13b180d Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 20 Jan 2025 22:08:08 +0800 Subject: [PATCH 14/31] [improve][broker] Remove spamming logs for customized managed ledger (#23862) (cherry picked from commit a93e93dddddc3751a2d29727fc5226b689fed204) (cherry picked from commit cae9f600ff17b8b2f563687f80971240de57246b) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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())) { From 043c557189c0918b9587726443aed1bfaffa78a1 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 20 Jan 2025 09:39:54 -0800 Subject: [PATCH 15/31] [improve] Upgrade to Netty 4.1.117.Final (#23863) (cherry picked from commit a96af729d805b679d0356b6dd2f6a1c66ad20252) (cherry picked from commit e287d3bb17796563b31128f95c97101bead4330e) --- buildtools/pom.xml | 2 +- .../server/src/assemble/LICENSE.bin.txt | 40 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 38 +++++++++--------- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 38 +++++++++--------- 5 files changed, 60 insertions(+), 60 deletions(-) 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/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-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index bcbb37aee2641..2cdf9c5bf2be5 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -231,21 +231,21 @@ The Apache Software License, Version 2.0 - commons-compress-1.26.0.jar - commons-lang3-3.11.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-haproxy-4.1.116.Final.jar - - netty-codec-socks-4.1.116.Final.jar - - netty-handler-proxy-4.1.116.Final.jar - - netty-common-4.1.116.Final.jar - - netty-handler-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-haproxy-4.1.117.Final.jar + - netty-codec-socks-4.1.117.Final.jar + - netty-handler-proxy-4.1.117.Final.jar + - netty-common-4.1.117.Final.jar + - netty-handler-4.1.117.Final.jar - netty-reactive-streams-2.0.6.jar - - netty-resolver-4.1.116.Final.jar - - netty-resolver-dns-4.1.116.Final.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-4.1.117.Final.jar + - netty-resolver-dns-4.1.117.Final.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 - 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 @@ -253,11 +253,11 @@ The Apache Software License, Version 2.0 - netty-tcnative-boringssl-static-2.0.69.Final-osx-x86_64.jar - netty-tcnative-boringssl-static-2.0.69.Final-windows-x86_64.jar - netty-tcnative-classes-2.0.69.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-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-incubator-transport-classes-io_uring-0.0.26.Final.jar - netty-incubator-transport-native-io_uring-0.0.26.Final-linux-x86_64.jar - netty-incubator-transport-native-io_uring-0.0.26.Final-linux-aarch_64.jar From 69e638a99539aced6f0de6fd54a6f9f6b72e8366 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 20 Jan 2025 10:42:51 -0800 Subject: [PATCH 16/31] [improve][broker] Improve Consumer.equals performance (#23864) (cherry picked from commit 3c2ec2bf8bfd94eded46b42c5089dd8321afd096) (cherry picked from commit 69caf71c7232bbb7db87c1dc3a5e3b5b7b615282) --- .../main/java/org/apache/pulsar/broker/service/Consumer.java | 3 +++ 1 file changed, 3 insertions(+) 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()); From 30e25e8b7eb37bd4d42101471b1ee272d65fd8f9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 22 Jan 2025 22:36:40 -0800 Subject: [PATCH 17/31] [improve][test] Add solution to PulsarMockBookKeeper for intercepting reads (#23875) (cherry picked from commit 87fb442c223d47d8a426b44575981345d7a23481) (cherry picked from commit 38f15bce242541ce33a97ff132d8cfacfee5c16e) --- .../client/PulsarMockBookKeeper.java | 8 +++- .../client/PulsarMockLedgerHandle.java | 2 +- .../client/PulsarMockReadHandle.java | 31 +++++++++----- .../PulsarMockReadHandleInterceptor.java | 40 +++++++++++++++++++ 4 files changed, 68 insertions(+), 13 deletions(-) create mode 100644 testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandleInterceptor.java diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java index 620b1c6fb6a2a..f249a0f7231cf 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -40,6 +40,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import lombok.Getter; +import lombok.Setter; import org.apache.bookkeeper.client.AsyncCallback.CreateCallback; import org.apache.bookkeeper.client.AsyncCallback.DeleteCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; @@ -96,6 +98,9 @@ public static Collection getMockEnsemble() { final Queue addEntryResponseDelaysMillis = new ConcurrentLinkedQueue<>(); final List> failures = new ArrayList<>(); final List> addEntryFailures = new ArrayList<>(); + @Setter + @Getter + private volatile PulsarMockReadHandleInterceptor readHandleInterceptor; public PulsarMockBookKeeper(OrderedExecutor orderedExecutor) throws Exception { this.orderedExecutor = orderedExecutor; @@ -250,7 +255,8 @@ public CompletableFuture execute() { return FutureUtils.exception(new BKException.BKUnauthorizedAccessException()); } else { return FutureUtils.value(new PulsarMockReadHandle(PulsarMockBookKeeper.this, ledgerId, - lh.getLedgerMetadata(), lh.entries)); + lh.getLedgerMetadata(), lh.entries, + PulsarMockBookKeeper.this::getReadHandleInterceptor)); } }); } diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java index aa61e541d0d6b..d30684e604670 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java @@ -73,7 +73,7 @@ public PulsarMockLedgerHandle(PulsarMockBookKeeper bk, long id, this.digest = digest; this.passwd = Arrays.copyOf(passwd, passwd.length); - readHandle = new PulsarMockReadHandle(bk, id, getLedgerMetadata(), entries); + readHandle = new PulsarMockReadHandle(bk, id, getLedgerMetadata(), entries, bk::getReadHandleInterceptor); } @Override diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandle.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandle.java index a4361f62254e4..9f3f4969199ce 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandle.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandle.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.LastConfirmedAndEntry; import org.apache.bookkeeper.client.api.LedgerEntries; @@ -40,28 +41,36 @@ class PulsarMockReadHandle implements ReadHandle { private final long ledgerId; private final LedgerMetadata metadata; private final List entries; + private final Supplier readHandleInterceptorSupplier; PulsarMockReadHandle(PulsarMockBookKeeper bk, long ledgerId, LedgerMetadata metadata, - List entries) { + List entries, + Supplier readHandleInterceptorSupplier) { this.bk = bk; this.ledgerId = ledgerId; this.metadata = metadata; this.entries = entries; + this.readHandleInterceptorSupplier = readHandleInterceptorSupplier; } @Override public CompletableFuture readAsync(long firstEntry, long lastEntry) { return bk.getProgrammedFailure().thenComposeAsync((res) -> { - log.debug("readEntries: first={} last={} total={}", firstEntry, lastEntry, entries.size()); - List seq = new ArrayList<>(); - long entryId = firstEntry; - while (entryId <= lastEntry && entryId < entries.size()) { - seq.add(entries.get((int) entryId++).duplicate()); - } - log.debug("Entries read: {}", seq); - - return FutureUtils.value(LedgerEntriesImpl.create(seq)); - }); + log.debug("readEntries: first={} last={} total={}", firstEntry, lastEntry, entries.size()); + List seq = new ArrayList<>(); + long entryId = firstEntry; + while (entryId <= lastEntry && entryId < entries.size()) { + seq.add(entries.get((int) entryId++).duplicate()); + } + log.debug("Entries read: {}", seq); + LedgerEntriesImpl ledgerEntries = LedgerEntriesImpl.create(seq); + PulsarMockReadHandleInterceptor pulsarMockReadHandleInterceptor = readHandleInterceptorSupplier.get(); + if (pulsarMockReadHandleInterceptor != null) { + return pulsarMockReadHandleInterceptor.interceptReadAsync(ledgerId, firstEntry, lastEntry, + ledgerEntries); + } + return FutureUtils.value(ledgerEntries); + }); } @Override diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandleInterceptor.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandleInterceptor.java new file mode 100644 index 0000000000000..acee87b0f77f4 --- /dev/null +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandleInterceptor.java @@ -0,0 +1,40 @@ +/* + * 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.client; + +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.client.api.LedgerEntries; + +/** + * Interceptor interface for intercepting read handle readAsync operations. + * This is useful for testing purposes, for example for introducing delays. + */ +public interface PulsarMockReadHandleInterceptor { + /** + * Intercepts the readAsync operation on a read handle. + * + * @param ledgerId ledger id + * @param firstEntry first entry to read + * @param lastEntry last entry to read + * @param entries entries that would be returned by the read operation + * @return CompletableFuture that will complete with the entries to return + */ + CompletableFuture interceptReadAsync(long ledgerId, long firstEntry, long lastEntry, + LedgerEntries entries); +} From 5d399d8df8b609a5ab812bf4b6d6860462bd0bf8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 22 Jan 2025 22:48:46 -0800 Subject: [PATCH 18/31] [fix][test] Fix quiet time implementation in BrokerTestUtil.receiveMessages (#23876) (cherry picked from commit 52e8730613c36008ea57a0ca5c10231512232d7e) (cherry picked from commit 720184d98e54cfccf1b96ed70c11c3ac9e6467a2) --- .../apache/pulsar/broker/BrokerTestUtil.java | 299 +++++++++++++++++- .../pulsar/broker/BrokerTestUtilTest.java | 115 +++++++ 2 files changed, 413 insertions(+), 1 deletion(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtilTest.java 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 From 87cb0eea20965e40751cb97cdf1757613cc8df39 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 23 Jan 2025 13:40:03 +0200 Subject: [PATCH 19/31] [improve][ci] Increase Maven max heap size to 2048M and tune GCLockerRetryAllocationCount (#23883) (cherry picked from commit 6d8d73df727cceb8db265d9e33455ee5c52a8791) (cherry picked from commit cde02acac7405dd6c01f99d3e1fdfaeca4a24539) --- .github/workflows/ci-owasp-dependency-check.yaml | 2 +- .github/workflows/pulsar-ci-flaky.yaml | 2 +- .github/workflows/pulsar-ci.yaml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/ci-owasp-dependency-check.yaml b/.github/workflows/ci-owasp-dependency-check.yaml index 6e60065b07ba9..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 }} diff --git a/.github/workflows/pulsar-ci-flaky.yaml b/.github/workflows/pulsar-ci-flaky.yaml index 7df18f59b880e..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 diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 42d840028cab7..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 From 1f1b7fcffb57bdd53cfef00f85371083346fde95 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 23 Jan 2025 16:02:45 +0200 Subject: [PATCH 20/31] [improve][broker] Support values up to 2^32 in ConcurrentBitmapSortedLongPairSet (#23878) (cherry picked from commit 0f9f66181fade7b5163372591d2b299b6ba31780) (cherry picked from commit 2e0d5752f08cb93ee65f576d4532d09a1745d9fe) --- .../ConcurrentBitmapSortedLongPairSet.java | 14 +++++++--- ...ConcurrentBitmapSortedLongPairSetTest.java | 26 ++++++++++++++++--- 2 files changed, 33 insertions(+), 7 deletions(-) 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/utils/ConcurrentBitmapSortedLongPairSetTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSetTest.java index 5f8f13288cfe8..91a7dcbede047 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSetTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSetTest.java @@ -18,18 +18,19 @@ */ package org.apache.pulsar.utils; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; -import lombok.Cleanup; -import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet; -import org.testng.annotations.Test; import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import lombok.Cleanup; +import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet; +import org.testng.annotations.Test; @Test(groups = "utils") public class ConcurrentBitmapSortedLongPairSetTest { @@ -204,4 +205,23 @@ public void concurrentInsertions() throws Throwable { assertEquals(set.size(), N * nThreads); } + + @Test + public void testValueLargerThanIntegerMAX_VALUE() { + ConcurrentBitmapSortedLongPairSet set = new ConcurrentBitmapSortedLongPairSet(); + long baseValue = Integer.MAX_VALUE; + List addedValues = new ArrayList<>(); + int items = 10; + for (int i = 0; i < items; i++) { + long value = baseValue + i; + set.add(1, value); + addedValues.add(value); + } + assertEquals(set.size(), items); + Set values = set.items(items, (item1, item2) -> { + assertEquals(item1, 1); + return item2; + }); + assertThat(values).containsExactlyInAnyOrderElementsOf(addedValues); + } } From 774cfd35c75913958cdebaf2bd19ee22063e49e4 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 1 Oct 2024 12:53:45 -0700 Subject: [PATCH 21/31] [fix][broker] Support large number of unack message store for cursor recovery (#9292) (cherry picked from commit 9eeffe595b6c2312b1b92eb8b9606639f25ab276) (cherry picked from commit 8df09321b4e3d6d92e1bda6160518fff899db8f3) --- .../mledger/ManagedLedgerConfig.java | 4 +- .../mledger/impl/ManagedCursorImpl.java | 64 +++++++++++++++++-- .../mledger/impl/RangeSetWrapper.java | 29 +++++++++ .../src/main/proto/MLDataFormats.proto | 6 ++ .../mledger/impl/ManagedCursorTest.java | 11 ++-- .../mledger/impl/ManagedLedgerBkTest.java | 47 +++++++++++++- .../util/collections/LongPairRangeSet.java | 14 ++++ .../collections/OpenLongPairRangeSet.java | 46 +++++++++++++ 8 files changed, 207 insertions(+), 14 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index fe8cb238dbce4..20c48b2441960 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 @@ -505,8 +505,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/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index c10ae2fbad343..bcae8c04d1440 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 @@ -60,6 +60,7 @@ 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; @@ -92,12 +93,15 @@ 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.LongPairRangeSet; @@ -601,9 +605,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()); @@ -622,6 +624,45 @@ 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())); + individualDeletedMessages.build(rangeMap); + } 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 { @@ -3091,12 +3132,23 @@ 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; + 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(), 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..11cce409bec54 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 @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; @@ -142,6 +143,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 +187,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/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/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..6d8f89e595385 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 @@ -23,7 +23,6 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -import io.netty.buffer.ByteBuf; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -34,7 +33,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import lombok.Cleanup; + import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperTestClient; import org.apache.bookkeeper.client.api.DigestType; @@ -53,9 +52,13 @@ 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.collections.LongPairRangeSet; import org.awaitility.Awaitility; import org.testng.annotations.Test; +import io.netty.buffer.ByteBuf; +import lombok.Cleanup; + public class ManagedLedgerBkTest extends BookKeeperClusterTestCase { public ManagedLedgerBkTest() { @@ -587,4 +590,44 @@ public void testPeriodicRollover() throws Exception { Awaitility.await().until(() -> cursorImpl.getCursorLedger() != currentLedgerId); } + /** + * This test validates that cursor serializes and deserializes individual-ack list from the bk-ledger. + * + * @throws Exception + */ + @Test + public void testUnackmessagesAndRecovery() throws Exception { + ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig(); + factoryConf.setMaxCacheSize(0); + + ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf); + + ManagedLedgerConfig config = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) + .setAckQuorumSize(1).setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1) + .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1); + ManagedLedger ledger = factory.open("my_test_unack_messages", config); + ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + + int totalEntries = 100; + for (int i = 0; i < totalEntries; i++) { + Position p = ledger.addEntry("entry".getBytes()); + if (i % 2 == 0) { + cursor.delete(p); + } + } + + LongPairRangeSet unackMessagesBefore = cursor.getIndividuallyDeletedMessagesSet(); + + ledger.close(); + + // open and recover cursor + ledger = factory.open("my_test_unack_messages", config); + cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + + LongPairRangeSet unackMessagesAfter = cursor.getIndividuallyDeletedMessagesSet(); + assertTrue(unackMessagesBefore.equals(unackMessagesAfter)); + + ledger.close(); + factory.shutdown(); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java index 8aad5587dfd38..df74857245bb3 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; import java.util.Set; import lombok.EqualsAndHashCode; @@ -136,6 +137,19 @@ public interface LongPairRangeSet> { */ Range lastRange(); + default Map toRanges(int maxRanges) { + throw new UnsupportedOperationException(); + } + + /** + * Build {@link LongPairRangeSet} using internal ranges returned by {@link #toRanges(int)} . + * + * @param ranges + */ + default void build(Map ranges) { + throw new UnsupportedOperationException(); + } + /** * Return the number bit sets to true from lower (inclusive) to upper (inclusive). */ diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java index 6df6d414871ec..3076c6c5c5fa1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java @@ -18,16 +18,21 @@ */ package org.apache.pulsar.common.util.collections; +import static java.util.BitSet.valueOf; import static java.util.Objects.requireNonNull; import com.google.common.collect.BoundType; import com.google.common.collect.Range; import java.util.ArrayList; import java.util.BitSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.NavigableMap; +import java.util.Objects; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.lang.mutable.MutableInt; @@ -250,6 +255,47 @@ public Range lastRange() { return Range.openClosed(consumer.apply(lastSet.getKey(), lower), consumer.apply(lastSet.getKey(), upper)); } + @Override + public Map toRanges(int maxRanges) { + Map internalBitSetMap = new HashMap<>(); + AtomicInteger rangeCount = new AtomicInteger(); + rangeBitSetMap.forEach((id, bmap) -> { + if (rangeCount.getAndAdd(bmap.cardinality()) > maxRanges) { + return; + } + internalBitSetMap.put(id, bmap.toLongArray()); + }); + return internalBitSetMap; + } + + @Override + public void build(Map internalRange) { + internalRange.forEach((id, ranges) -> { + BitSet bitset = createNewBitSet(); + bitset.or(valueOf(ranges)); + rangeBitSetMap.put(id, bitset); + }); + } + + + @Override + public int hashCode() { + return Objects.hashCode(rangeBitSetMap); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof OpenLongPairRangeSet)) { + return false; + } + if (this == obj) { + return true; + } + @SuppressWarnings("rawtypes") + OpenLongPairRangeSet set = (OpenLongPairRangeSet) obj; + return this.rangeBitSetMap.equals(set.rangeBitSetMap); + } + @Override public int cardinality(long lowerKey, long lowerValue, long upperKey, long upperValue) { NavigableMap subMap = rangeBitSetMap.subMap(lowerKey, true, upperKey, true); From 392174283200329e3513e5191662acb7f730cd3a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 20 Jan 2025 09:48:53 +0800 Subject: [PATCH 22/31] [fix] [ml] Fix cursor metadata compatability issue when switching the config unackedRangesOpenCacheSetEnabled (#23759) (cherry picked from commit 4ee463342c0aedff5c0f543e0cc17455590f3632) (cherry picked from commit 2609acd3e134dd4ff85cce7669e811c141099707) --- .../mledger/ManagedLedgerConfig.java | 7 + .../mledger/impl/ManagedCursorImpl.java | 48 ++++++- .../mledger/impl/RangeSetWrapper.java | 17 ++- .../mledger/impl/ManagedLedgerBkTest.java | 122 +++++++++++++++--- .../pulsar/broker/ServiceConfiguration.java | 4 + .../pulsar/broker/service/BrokerService.java | 2 + .../service/ManagedLedgerConfigTest.java | 70 ++++++++++ 7 files changed, 235 insertions(+), 35 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerConfigTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 20c48b2441960..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 */ 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 bcae8c04d1440..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 @@ -178,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; @@ -632,7 +633,22 @@ public void recoverIndividualDeletedMessages(PositionInfo positionInfo) { try { Map rangeMap = rangeList.stream().collect(Collectors.toMap(LongListMap::getKey, list -> list.getValuesList().stream().mapToLong(i -> i).toArray())); - individualDeletedMessages.build(rangeMap); + // 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); @@ -2335,7 +2351,14 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } // 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); @@ -3138,10 +3161,21 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin .addAllProperties(buildPropertiesMap(mdEntry.properties)); Map internalRanges = null; - try { - internalRanges = individualDeletedMessages.toRanges(getConfig().getMaxUnackedRangesToPersist()); - } catch (Exception e) { - log.warn("[{}]-{} Failed to serialize individualDeletedMessages", ledger.getName(), name, e); + /** + * 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)); 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 11cce409bec54..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,14 +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 java.util.Map; -import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; import org.roaringbitmap.RoaringBitSet; @@ -40,7 +38,6 @@ public class RangeSetWrapper> implements LongPairRangeSe private final LongPairRangeSet rangeSet; private final LongPairConsumer rangeConverter; - private final ManagedLedgerConfig config; private final boolean enableMultiEntry; /** @@ -53,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 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 6d8f89e595385..8ae5a55fea793 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,13 +20,16 @@ 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 java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; +import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.Future; @@ -34,8 +37,10 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +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.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; @@ -49,18 +54,23 @@ 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.collections.LongPairRangeSet; +import org.awaitility.Awaitility; +import org.testng.annotations.DataProvider; import org.awaitility.Awaitility; import org.testng.annotations.Test; import io.netty.buffer.ByteBuf; import lombok.Cleanup; +@Slf4j public class ManagedLedgerBkTest extends BookKeeperClusterTestCase { + private final ObjectMapper jackson = new ObjectMapper(); + public ManagedLedgerBkTest() { super(2); } @@ -590,44 +600,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 - public void testUnackmessagesAndRecovery() 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(); - factoryConf.setMaxCacheSize(0); - ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf); - - ManagedLedgerConfig config = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) + 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); - ManagedLedger ledger = factory.open("my_test_unack_messages", config); - ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + .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 = ledger.addEntry("entry".getBytes()); + Position p = ledger1.addEntry("entry".getBytes()); if (i % 2 == 0) { - cursor.delete(p); + cursor1.delete(p); } } + log.info("ack ranges: {}", cursor1.getIndividuallyDeletedMessagesSet().size()); - LongPairRangeSet unackMessagesBefore = cursor.getIndividuallyDeletedMessagesSet(); + // reopen and recover cursor + ledger1.close(); + ManagedLedger ledger2 = factory.open(mlName, config2); + ManagedCursorImpl cursor2 = (ManagedCursorImpl) ledger2.openCursor(cursorName); - ledger.close(); + log.info("before: {}", cursor1.getIndividuallyDeletedMessagesSet().asRanges()); + log.info("after : {}", cursor2.getIndividuallyDeletedMessagesSet().asRanges()); + assertEquals(cursor1.getIndividuallyDeletedMessagesSet().asRanges(), cursor2.getIndividuallyDeletedMessagesSet().asRanges()); + assertEquals(cursor1.markDeletePosition, cursor2.markDeletePosition); - // open and recover cursor - ledger = factory.open("my_test_unack_messages", config); - cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + ledger2.close(); + factory.shutdown(); + } - LongPairRangeSet unackMessagesAfter = cursor.getIndividuallyDeletedMessagesSet(); - assertTrue(unackMessagesBefore.equals(unackMessagesAfter)); + @DataProvider(name = "booleans") + public Object[][] booleans() { + return new Object[][] { + {true}, + {false}, + }; + } - ledger.close(); + @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/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..57ab358c7ad3f 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 @@ -2146,6 +2146,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/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/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); + } +} + From 159b721d66ee261ae49acc2c8bf4b37fecfafffe Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Sat, 25 Jan 2025 02:44:16 +0800 Subject: [PATCH 23/31] [fix][client] Fix LoadManagerReport not found (#23886) (cherry picked from commit 223eea027beb1a7e95ff4519db181268b4636829) (cherry picked from commit da4db5e8c9153197e86598a72595478f3994226e) --- pulsar-client-admin-shaded/pom.xml | 7 +++++++ pulsar-client-all/pom.xml | 7 +++++++ pulsar-client-shaded/pom.xml | 7 +++++++ .../pulsar/common/util/ObjectMapperFactory.java | 11 +---------- .../apache/pulsar/tests/integration/SmokeTest.java | 1 + .../apache/pulsar/tests/integration/SmokeTest.java | 1 + 6 files changed, 24 insertions(+), 10 deletions(-) diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 15538d8a4ecf7..179e48436964b 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -348,6 +348,13 @@ org.apache.pulsar.policies org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport + org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats + org.apache.pulsar.policies.data.loadbalancer.ResourceUsage + org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData + org.asynchttpclient diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index beccabe92e7ff..caba8a368bfb0 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -405,6 +405,13 @@ org.apache.pulsar.policies org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport + org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats + org.apache.pulsar.policies.data.loadbalancer.ResourceUsage + org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData + org.asynchttpclient diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index 26ab43a3a5c3b..a49c96cd49587 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -318,6 +318,13 @@ org.apache.pulsar.policies org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport + org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats + org.apache.pulsar.policies.data.loadbalancer.ResourceUsage + org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData + org.asynchttpclient diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java index 7b235cfa341d1..cd88ccdd4ef41 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java @@ -32,7 +32,6 @@ import com.fasterxml.jackson.module.paramnames.ParameterNamesModule; import java.util.concurrent.atomic.AtomicReference; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.ClassUtils; import org.apache.pulsar.client.admin.internal.data.AuthPoliciesImpl; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.FunctionState; @@ -260,15 +259,7 @@ private static void setAnnotationsModule(ObjectMapper mapper) { mapper.addMixIn(FunctionState.class, JsonIgnorePropertiesMixIn.class); mapper.addMixIn(Metrics.class, MetricsMixIn.class); - try { - // We look for LoadManagerReport first, then add deserializer to the module - // With shaded client, org.apache.pulsar.policies is relocated to - // org.apache.pulsar.shade.org.apache.pulsar.policies - ClassUtils.getClass("org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport"); - module.addDeserializer(LoadManagerReport.class, new LoadReportDeserializer()); - } catch (ClassNotFoundException e) { - log.debug("Add LoadManagerReport deserializer failed because LoadManagerReport.class has been shaded", e); - } + module.addDeserializer(LoadManagerReport.class, new LoadReportDeserializer()); module.setAbstractTypes(resolver); diff --git a/tests/pulsar-client-admin-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java b/tests/pulsar-client-admin-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java index 990a0a8211c71..3dde44b2e911a 100644 --- a/tests/pulsar-client-admin-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java +++ b/tests/pulsar-client-admin-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java @@ -86,6 +86,7 @@ public void checkAdmin() throws PulsarClientException, PulsarAdminException { expectedNamespacesList.add("public/default"); expectedNamespacesList.add("public/functions"); Assert.assertEquals(admin.namespaces().getNamespaces("public"), expectedNamespacesList); + admin.brokerStats().getLoadReport(); } @Override diff --git a/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java b/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java index e5eef415a513d..6b1837f5b8ee5 100644 --- a/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java +++ b/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java @@ -86,6 +86,7 @@ public void checkAdmin() throws PulsarClientException, PulsarAdminException { expectedNamespacesList.add("public/default"); expectedNamespacesList.add("public/functions"); Assert.assertEquals(admin.namespaces().getNamespaces("public"), expectedNamespacesList); + admin.brokerStats().getLoadReport(); } @Override From d48c0a6a4deb44740f8c37a302fc5c75954ffafb Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 28 Jan 2025 03:26:32 +0800 Subject: [PATCH 24/31] [fix][broker] Fix repeatedly acquired pending reads quota (#23869) (cherry picked from commit 331a997b76b83b3eca777c4559eb60b940d30c27) (cherry picked from commit 0132d930cf4c2c6777220163d175b2456a4fad02) --- .../impl/cache/PendingReadsManager.java | 8 +- .../impl/cache/RangeEntryCacheImpl.java | 26 +- .../InflightReadsLimiterIntegrationTest.java | 231 ++++++++++++++++++ .../impl/cache/PendingReadsManagerTest.java | 2 +- 4 files changed, 251 insertions(+), 16 deletions(-) create mode 100644 managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java 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..d733b54dd1304 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 @@ -362,7 +362,7 @@ public void readEntriesFailed(ManagedLedgerException exception, }; rangeEntryCache.asyncReadEntry0(lh, missingOnRight.startEntry, missingOnRight.endEntry, - shouldCacheEntry, readFromRightCallback, null); + shouldCacheEntry, readFromRightCallback, null, false); } @Override @@ -372,7 +372,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 +395,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 +418,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..f378acfba14a8 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 @@ -56,7 +56,7 @@ 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 final RangeEntryCacheManagerImpl manager; final ManagedLedgerImpl ml; @@ -101,7 +101,7 @@ public String getName() { } @VisibleForTesting - InflightReadsLimiter getPendingReadsLimiter() { + public InflightReadsLimiter getPendingReadsLimiter() { return manager.getInflightReadsLimiter(); } @@ -281,7 +281,7 @@ private void asyncReadEntry0(ReadHandle lh, PositionImpl position, final ReadEnt 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,16 +294,20 @@ 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 withLimits) { + asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx, null, withLimits); } void asyncReadEntry0WithLimits(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry, - final ReadEntriesCallback originalCallback, Object ctx, InflightReadsLimiter.Handle handle) { - - final AsyncCallbacks.ReadEntriesCallback callback = - handlePendingReadsLimits(lh, firstEntry, lastEntry, shouldCacheEntry, - originalCallback, ctx, handle); + final ReadEntriesCallback originalCallback, Object ctx, InflightReadsLimiter.Handle handle, + boolean withLimits) { + AsyncCallbacks.ReadEntriesCallback callback; + if (withLimits) { + callback = handlePendingReadsLimits(lh, firstEntry, lastEntry, shouldCacheEntry, originalCallback, ctx, + handle); + } else { + callback = originalCallback; + } if (callback == null) { return; } @@ -381,7 +385,7 @@ private AsyncCallbacks.ReadEntriesCallback handlePendingReadsLimits(ReadHandle l } ml.getExecutor().execute(() -> { asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry, - originalCallback, ctx, newHandle); + originalCallback, ctx, newHandle, true); }); return null; } else { 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..b57dea6a5bb4d --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java @@ -0,0 +1,231 @@ +/* + * 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.awaitility.reflect.WhiteboxImpl; +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 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); + Assert.assertEquals(sizePerEntry1, 1); + 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 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); + Assert.assertEquals(sizePerEntry2, 1); + 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 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); + Assert.assertEquals(sizePerEntry3, 1); + 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/cache/PendingReadsManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java index 6f573ff8d75c8..75e371fa97c2a 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 @@ -110,7 +110,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); From cab5f06b9268768b3bef566cc90a7f8dfed32978 Mon Sep 17 00:00:00 2001 From: Philipp Dolif <52791955+pdolif@users.noreply.github.com> Date: Tue, 28 Jan 2025 23:17:21 +0100 Subject: [PATCH 25/31] [fix][test] Fix flaky DelayedDeliveryTest.testEnableTopicDelayedDelivery (#23893) (cherry picked from commit 9079262aaedce719bda786a63781292a4257e299) (cherry picked from commit f9d3dbf24513cdd9e853e029e300ee9ac2e40132) --- .../pulsar/broker/service/persistent/DelayedDeliveryTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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); } From 8e8af6c88281d721bbd71d6f3e0f129614166749 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 29 Jan 2025 01:23:52 +0200 Subject: [PATCH 26/31] [fix][broker Fix bug in RangeCache where different instance of the key wouldn't ever match (#23903) (cherry picked from commit b6cfecce5f3a1eecbf6f5df81cb835fbbfe35980) (cherry picked from commit c11ebe16e2f59dda15cc2f74af770fedc6fa9b89) --- .../bookkeeper/mledger/util/RangeCache.java | 109 +++++-- .../mledger/impl/ManagedLedgerBkTest.java | 115 ++++++- .../mledger/util/RangeCacheTest.java | 19 +- .../auth/MockedPulsarServiceBaseTest.java | 5 + .../KeySharedSubscriptionBrokerCacheTest.java | 285 ++++++++++++++++++ pulsar-broker/src/test/resources/log4j2.xml | 9 + .../impl/TxnLogBufferedWriterTest.java | 9 + .../test/MockedBookKeeperTestCase.java | 6 +- 8 files changed, 525 insertions(+), 32 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionBrokerCacheTest.java 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/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java index 8ae5a55fea793..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 @@ -24,24 +24,28 @@ 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; @@ -58,14 +62,12 @@ 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; -import io.netty.buffer.ByteBuf; -import lombok.Cleanup; - @Slf4j public class ManagedLedgerBkTest extends BookKeeperClusterTestCase { @@ -242,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") 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/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/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/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml index 09a89702ee2ac..b0d2e8af255c7 100644 --- a/pulsar-broker/src/test/resources/log4j2.xml +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -36,5 +36,14 @@ + + diff --git a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java index 3c6b2e382e311..efe4305d2daa8 100644 --- a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java +++ b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java @@ -52,6 +52,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -126,6 +127,14 @@ public Object[][] mainProcessCasesProvider(){ }; } + @Override + protected ManagedLedgerFactoryConfig createManagedLedgerFactoryConfig() { + ManagedLedgerFactoryConfig managedLedgerFactoryConfig = super.createManagedLedgerFactoryConfig(); + // disable the broker cache so that assertAllByteBufHasBeenReleased can work correctly. + managedLedgerFactoryConfig.setMaxCacheSize(0); + return managedLedgerFactoryConfig; + } + /** * Tests all operations from write to callback, including these step: * 1. Write many data. diff --git a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/test/MockedBookKeeperTestCase.java b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/test/MockedBookKeeperTestCase.java index ac5aa3bd8927e..e3e6945620c34 100644 --- a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/test/MockedBookKeeperTestCase.java +++ b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/test/MockedBookKeeperTestCase.java @@ -82,10 +82,14 @@ public void setUp(Method method) throws Exception { throw e; } - ManagedLedgerFactoryConfig conf = new ManagedLedgerFactoryConfig(); + ManagedLedgerFactoryConfig conf = createManagedLedgerFactoryConfig(); factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, conf); } + protected ManagedLedgerFactoryConfig createManagedLedgerFactoryConfig() { + return new ManagedLedgerFactoryConfig(); + } + @AfterMethod(alwaysRun = true) public void tearDown(Method method) { try { From f58ea21d5f677056b54567ff566bed5899744897 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 29 Jan 2025 19:33:40 +0200 Subject: [PATCH 27/31] [fix][ci] Configure Docker data-root to /mnt/docker to avoid running out of disk space (#23909) (cherry picked from commit ed5dbb5289f09f913f07a38ba1481727f00f063a) (cherry picked from commit 9804170a733affd5121c9270d43a2599c12d3dc1) --- .github/actions/clean-disk/action.yml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/.github/actions/clean-disk/action.yml b/.github/actions/clean-disk/action.yml index d74c3f25fc64c..e67ce59a08ddb 100644 --- a/.github/actions/clean-disk/action.yml +++ b/.github/actions/clean-disk/action.yml @@ -46,6 +46,15 @@ runs: time df -BM / /mnt echo "::endgroup::" done + if [[ "${{ inputs.mode }}" == "full" ]]; then + echo "::group::Moving /var/lib/docker to /mnt/docker" + sudo systemctl stop docker + echo '{"data-root": "/mnt/docker"}' | sudo tee /etc/docker/daemon.json + sudo mv /var/lib/docker /mnt/docker + sudo systemctl start docker + time df -BM / /mnt + echo "::endgroup::" + fi echo "::group::Cleaning apt state" time sudo bash -c "apt-get clean; apt-get autoclean; apt-get -y --purge autoremove" time df -BM / /mnt From bc5a020c62e51c8e80b071b8baee5a51e16ac4ac Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 29 Jan 2025 22:11:56 +0200 Subject: [PATCH 28/31] [fix][broker] Make InflightReadsLimiter asynchronous and apply it for replay queue reads (#23901) (cherry picked from commit c5173d5e15efade90afb9b0b1c19f3ba5b3aab37) (cherry picked from commit 37f0bc22dffe6468fc775d68ad1a9c7f9516cf3a) --- .../mledger/ManagedLedgerFactoryConfig.java | 12 + .../impl/ManagedLedgerFactoryImpl.java | 2 +- .../impl/cache/InflightReadsLimiter.java | 250 ++++++-- .../impl/cache/PendingReadsManager.java | 156 +++-- .../impl/cache/RangeEntryCacheImpl.java | 298 ++++----- .../cache/RangeEntryCacheManagerImpl.java | 15 +- .../InflightReadsLimiterIntegrationTest.java | 13 +- .../mledger/impl/ManagedLedgerTest.java | 92 +-- .../impl/cache/InflightReadsLimiterTest.java | 577 ++++++++++++++---- .../impl/cache/PendingReadsManagerTest.java | 4 +- .../pulsar/broker/ServiceConfiguration.java | 9 + .../broker/ManagedLedgerClientFactory.java | 17 +- 12 files changed, 992 insertions(+), 453 deletions(-) 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/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/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 d733b54dd1304..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; 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 f378acfba14a8..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; @@ -57,6 +58,8 @@ public class RangeEntryCacheImpl implements EntryCache { * Overhead per-entry to take into account the envelope. */ 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; @@ -117,17 +118,18 @@ public 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,47 +256,6 @@ 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) { @@ -294,38 +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, boolean withLimits) { - asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx, null, withLimits); + 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, - boolean withLimits) { - AsyncCallbacks.ReadEntriesCallback callback; - if (withLimits) { - callback = handlePendingReadsLimits(lh, firstEntry, lastEntry, shouldCacheEntry, originalCallback, ctx, - 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()); + + if (log.isDebugEnabled()) { + log.debug("[{}] Reading {} entries in range {} to {}", ml.getName(), numberOfEntries, firstPosition, + lastPosition); + } + + InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); + if (!acquirePermits || pendingReadsLimiter.isDisabled()) { + doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, shouldCacheEntry, + originalCallback, ctx); } else { - callback = originalCallback; + 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 (callback == null) { + } + + 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); + } - 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); + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx2) { + pendingReadsLimiter.release(handle); + originalCallback.readEntriesFailed(exception, ctx2); + } + }; + doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, shouldCacheEntry, + wrappedCallback, ctx); + } - if (log.isDebugEnabled()) { - log.debug("[{}] Reading entries range ledger {}: {} to {}", ml.getName(), ledgerId, firstEntry, lastEntry); + 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); } - Collection 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) { @@ -336,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()) { @@ -348,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, true); - }); - 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; } /** @@ -441,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/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java index b57dea6a5bb4d..48f0cf08ddff4 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java @@ -40,7 +40,6 @@ import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.awaitility.Awaitility; -import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; import org.mockito.stubbing.Answer; import org.testng.Assert; @@ -142,8 +141,8 @@ public void testPreciseLimitation(String missingCase) throws Exception { SimpleReadEntriesCallback cb0 = new SimpleReadEntriesCallback(); entryCache.asyncReadEntry(spyCurrentLedger, 125, 125, true, cb0, ctx); cb0.entries.join(); - Long sizePerEntry1 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); - Assert.assertEquals(sizePerEntry1, 1); + 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); @@ -179,8 +178,8 @@ public void testPreciseLimitation(String missingCase) throws Exception { Thread.sleep(3000); readCompleteSignal1.countDown(); cb1.entries.join(); - Long sizePerEntry2 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); - Assert.assertEquals(sizePerEntry2, 1); + 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); @@ -192,8 +191,8 @@ public void testPreciseLimitation(String missingCase) throws Exception { readCompleteSignal2.countDown(); cb2.entries.join(); - Long sizePerEntry3 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); - Assert.assertEquals(sizePerEntry3, 1); + 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); 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 75e371fa97c2a..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() { 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 57ab358c7ad3f..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, 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()); From fb116923663b1ecfcc9dd25157a87001feb7fabc Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 29 Jan 2025 18:28:39 -0800 Subject: [PATCH 29/31] [fix] Avoid NPE when closing an uninitialized SameAuthParamsLookupAutoClusterFailover (#23911) (cherry picked from commit 12b057999f5674ae6da6b04ff8bd212d1904cc88) (cherry picked from commit c0ee98928c87828da4135b1b2df4babf00c7eb76) --- .../pulsar/client/impl/PulsarClientImpl.java | 3 +++ .../SameAuthParamsLookupAutoClusterFailover.java | 14 ++++++++++++-- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 52583ce0afeff..9325d6b45653e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -253,6 +253,9 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG this::reduceConsumerReceiverQueueSize); state.set(State.Open); } catch (Throwable t) { + // Log the exception first, or it could be missed if there are any subsequent exceptions in the + // shutdown sequence + log.error("Failed to create Pulsar client instance.", t); shutdown(); shutdownEventLoopGroup(eventLoopGroupReference); closeCnxPool(connectionPoolReference); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java index 1b7013f3018b8..9e96932a69608 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java @@ -107,11 +107,21 @@ public String getServiceUrl() { @Override public void close() throws Exception { + if (closed) { + return; + } + log.info("Closing service url provider. Current pulsar service: [{}] {}", currentPulsarServiceIndex, pulsarServiceUrlArray[currentPulsarServiceIndex]); + if (scheduledCheckTask != null) { + scheduledCheckTask.cancel(false); + } + + if (executor != null) { + executor.shutdownNow(); + } + closed = true; - scheduledCheckTask.cancel(false); - executor.shutdownNow(); } private int firstHealthyPulsarService() { From d84d9a472c497d71ef55eaec9da338442e16f371 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 30 Jan 2025 18:23:37 -0800 Subject: [PATCH 30/31] [fix] Initialize UrlServiceProvider before trying to use transaction coordinator (#23914) (cherry picked from commit 0e3b46e7e7445a2a6983c57f5cbc64fc1369ffad) --- .../org/apache/pulsar/client/impl/ClientBuilderImpl.java | 6 +----- .../org/apache/pulsar/client/impl/PulsarClientImpl.java | 4 ++++ 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java index f49b704445fbd..a9e42de192279 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java @@ -64,11 +64,7 @@ public PulsarClient build() throws PulsarClientException { if (conf.getAuthentication() == null || conf.getAuthentication() == AuthenticationDisabled.INSTANCE) { setAuthenticationFromPropsIfAvailable(conf); } - PulsarClient client = new PulsarClientImpl(conf); - if (conf.getServiceUrlProvider() != null) { - conf.getServiceUrlProvider().initialize(client); - } - return client; + return new PulsarClientImpl(conf); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 9325d6b45653e..c5ec93bcdd2a8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -238,6 +238,10 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG this.timer = timer; } + if (conf.getServiceUrlProvider() != null) { + conf.getServiceUrlProvider().initialize(this); + } + if (conf.isEnableTransaction()) { tcClient = new TransactionCoordinatorClientImpl(this); try { From e1b048494459a43c81ade5e8db27ddd7458b5cf4 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Mon, 3 Feb 2025 15:16:48 +0530 Subject: [PATCH 31/31] Revert "[fix][ci] Configure Docker data-root to /mnt/docker to avoid running out of disk space (#23909)" This reverts commit f58ea21d5f677056b54567ff566bed5899744897. --- .github/actions/clean-disk/action.yml | 9 --------- 1 file changed, 9 deletions(-) diff --git a/.github/actions/clean-disk/action.yml b/.github/actions/clean-disk/action.yml index e67ce59a08ddb..d74c3f25fc64c 100644 --- a/.github/actions/clean-disk/action.yml +++ b/.github/actions/clean-disk/action.yml @@ -46,15 +46,6 @@ runs: time df -BM / /mnt echo "::endgroup::" done - if [[ "${{ inputs.mode }}" == "full" ]]; then - echo "::group::Moving /var/lib/docker to /mnt/docker" - sudo systemctl stop docker - echo '{"data-root": "/mnt/docker"}' | sudo tee /etc/docker/daemon.json - sudo mv /var/lib/docker /mnt/docker - sudo systemctl start docker - time df -BM / /mnt - echo "::endgroup::" - fi echo "::group::Cleaning apt state" time sudo bash -c "apt-get clean; apt-get autoclean; apt-get -y --purge autoremove" time df -BM / /mnt