From 78b843dc658e175ed5134d8a903cdd0f8055a492 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 6 Feb 2025 10:14:28 +0800 Subject: [PATCH 01/44] [fix][broker] Closed topics won't be removed from the cache (#23884) (cherry picked from commit 8a40b30cf47a91ec02d931e6371d02409ba5951e) (cherry picked from commit 5d35d41a09185bdd631826247b7bfb495209f847) --- .../apache/pulsar/broker/PulsarService.java | 2 +- .../pulsar/broker/service/AbstractTopic.java | 8 + .../pulsar/broker/service/BrokerService.java | 51 ++---- .../buffer/impl/TopicTransactionBuffer.java | 21 ++- .../impl/TransactionPersistentTopicTest.java | 148 ++++++++++++++++++ .../client/api/OrphanPersistentTopicTest.java | 4 +- 6 files changed, 184 insertions(+), 50 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionPersistentTopicTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 8252868266d1d..f22fb66739a6b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1977,7 +1977,7 @@ protected BrokerService newBrokerService(PulsarService pulsar) throws Exception } @VisibleForTesting - public void setTransactionExecutorProvider(TransactionBufferProvider transactionBufferProvider) { + public void setTransactionBufferProvider(TransactionBufferProvider transactionBufferProvider) { this.transactionBufferProvider = transactionBufferProvider; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index 9efef66fc3d74..11f36602d5fe2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -43,6 +43,7 @@ import java.util.function.ToLongFunction; import javax.annotation.Nonnull; import lombok.Getter; +import lombok.Setter; import org.apache.bookkeeper.mledger.util.StatsBuckets; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; @@ -90,6 +91,13 @@ public abstract class AbstractTopic implements Topic, TopicPolicyListener> createFuture; + // Producers currently connected to this topic protected final ConcurrentHashMap producers; 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 ca53ab0fc3183..4f88baf6f618b 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 @@ -1327,6 +1327,7 @@ private CompletableFuture> createNonPersistentTopic(String topic NonPersistentTopic nonPersistentTopic; try { nonPersistentTopic = newTopic(topic, null, this, NonPersistentTopic.class); + nonPersistentTopic.setCreateFuture(topicFuture); } catch (Throwable e) { log.warn("Failed to create topic {}", topic, e); topicFuture.completeExceptionally(e); @@ -1769,6 +1770,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { PersistentTopic persistentTopic = isSystemTopic(topic) ? new SystemTopic(topic, ledger, BrokerService.this) : newTopic(topic, ledger, BrokerService.this, PersistentTopic.class); + persistentTopic.setCreateFuture(topicFuture); persistentTopic .initialize() .thenCompose(__ -> persistentTopic.preCreateSubscriptionForCompactionIfNeeded()) @@ -2375,47 +2377,18 @@ public AuthorizationService getAuthorizationService() { return authorizationService; } - public CompletableFuture removeTopicFromCache(Topic topic) { - Optional>> createTopicFuture = findTopicFutureInCache(topic); - if (createTopicFuture.isEmpty()){ - return CompletableFuture.completedFuture(null); - } - return removeTopicFutureFromCache(topic.getName(), createTopicFuture.get()); - } - - private Optional>> findTopicFutureInCache(Topic topic){ - if (topic == null){ - return Optional.empty(); - } - final CompletableFuture> createTopicFuture = topics.get(topic.getName()); - // If not exists in cache, do nothing. - if (createTopicFuture == null){ - return Optional.empty(); - } - // If the future in cache is not yet complete, the topic instance in the cache is not the same with the topic. - if (!createTopicFuture.isDone()){ - return Optional.empty(); - } - // If the future in cache has exception complete, - // the topic instance in the cache is not the same with the topic. - if (createTopicFuture.isCompletedExceptionally()){ - return Optional.empty(); - } - Optional optionalTopic = createTopicFuture.join(); - Topic topicInCache = optionalTopic.orElse(null); - if (topicInCache == null || topicInCache != topic){ - return Optional.empty(); - } else { - return Optional.of(createTopicFuture); - } - } - - private CompletableFuture removeTopicFutureFromCache(String topic, - CompletableFuture> createTopicFuture) { - TopicName topicName = TopicName.get(topic); + /** + * Removes the topic from the cache only if the topicName and associated createFuture match exactly. + * The TopicEvent.UNLOAD event will be triggered before and after removal. + * + * @param topic The topic to be removed. + * @return A CompletableFuture that completes when the operation is done. + */ + public CompletableFuture removeTopicFromCache(AbstractTopic topic) { + TopicName topicName = TopicName.get(topic.getName()); return pulsar.getNamespaceService().getBundleAsync(topicName) .thenAccept(namespaceBundle -> { - removeTopicFromCache(topic, namespaceBundle, createTopicFuture); + removeTopicFromCache(topic.getName(), namespaceBundle, topic.getCreateFuture()); }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 04053e168e92a..58a31a4d2ee27 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -103,9 +103,20 @@ public class TopicTransactionBuffer extends TopicTransactionBufferState implemen private final AbortedTxnProcessor snapshotAbortedTxnProcessor; private final MaxReadPositionCallBack maxReadPositionCallBack; - private final AbortedTxnProcessor.SnapshotType snapshotType; + + private static AbortedTxnProcessor createSnapshotProcessor(PersistentTopic topic) { + return topic.getBrokerService().getPulsar().getConfiguration().isTransactionBufferSegmentedSnapshotEnabled() + ? new SnapshotSegmentAbortedTxnProcessorImpl(topic) + : new SingleSnapshotAbortedTxnProcessorImpl(topic); + } + public TopicTransactionBuffer(PersistentTopic topic) { + this(topic, createSnapshotProcessor(topic)); + } + + @VisibleForTesting + TopicTransactionBuffer(PersistentTopic topic, AbortedTxnProcessor snapshotAbortedTxnProcessor) { super(State.None); this.topic = topic; this.timer = topic.getBrokerService().getPulsar().getTransactionTimer(); @@ -114,13 +125,7 @@ public TopicTransactionBuffer(PersistentTopic topic) { this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar() .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis(); this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); - if (topic.getBrokerService().getPulsar().getConfiguration().isTransactionBufferSegmentedSnapshotEnabled()) { - snapshotAbortedTxnProcessor = new SnapshotSegmentAbortedTxnProcessorImpl(topic); - snapshotType = AbortedTxnProcessor.SnapshotType.Segment; - } else { - snapshotAbortedTxnProcessor = new SingleSnapshotAbortedTxnProcessorImpl(topic); - snapshotType = AbortedTxnProcessor.SnapshotType.Single; - } + this.snapshotAbortedTxnProcessor = snapshotAbortedTxnProcessor; this.maxReadPositionCallBack = topic.getMaxReadPositionCallBack(); this.recover(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionPersistentTopicTest.java new file mode 100644 index 0000000000000..8538ea2bd9984 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionPersistentTopicTest.java @@ -0,0 +1,148 @@ +/* + * 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.transaction.buffer.impl; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import java.io.IOException; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.TopicFactory; +import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; +import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class TransactionPersistentTopicTest extends ProducerConsumerBase { + + private static CountDownLatch topicInitSuccessSignal = new CountDownLatch(1); + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + // Intercept when the `topicFuture` is about to complete and wait until the topic close operation finishes. + conf.setTopicFactoryClassName(MyTopicFactory.class.getName()); + conf.setTransactionCoordinatorEnabled(true); + conf.setBrokerDeduplicationEnabled(false); + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testNoOrphanClosedTopicIfTxnInternalFailed() { + String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp2"); + + BrokerService brokerService = pulsar.getBrokerService(); + + // 1. Mock close topic when create transactionBuffer + TransactionBufferProvider mockTransactionBufferProvider = originTopic -> { + AbortedTxnProcessor abortedTxnProcessor = mock(AbortedTxnProcessor.class); + doAnswer(invocation -> { + topicInitSuccessSignal.await(); + return CompletableFuture.failedFuture(new RuntimeException("Mock recovery failed")); + }).when(abortedTxnProcessor).recoverFromSnapshot(); + when(abortedTxnProcessor.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); + return new TopicTransactionBuffer( + (PersistentTopic) originTopic, abortedTxnProcessor); + }; + TransactionBufferProvider originalTransactionBufferProvider = pulsar.getTransactionBufferProvider(); + pulsar.setTransactionBufferProvider(mockTransactionBufferProvider); + + // 2. Trigger create topic and assert topic load success. + CompletableFuture> firstLoad = brokerService.getTopic(tpName, true); + Awaitility.await().ignoreExceptions().atMost(10, TimeUnit.SECONDS) + .pollInterval(200, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertTrue(firstLoad.isDone()); + assertFalse(firstLoad.isCompletedExceptionally()); + }); + + // 3. Assert topic removed from cache + Awaitility.await().ignoreExceptions().atMost(10, TimeUnit.SECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertFalse(brokerService.getTopics().containsKey(tpName)); + }); + + // 4. Set txn provider to back + pulsar.setTransactionBufferProvider(originalTransactionBufferProvider); + } + + public static class MyTopicFactory implements TopicFactory { + @Override + public T create(String topic, ManagedLedger ledger, BrokerService brokerService, + Class topicClazz) { + try { + if (topicClazz == NonPersistentTopic.class) { + return (T) new NonPersistentTopic(topic, brokerService); + } else { + return (T) new MyPersistentTopic(topic, ledger, brokerService); + } + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + + @Override + public void close() throws IOException { + // No-op + } + } + + public static class MyPersistentTopic extends PersistentTopic { + + public MyPersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerService) { + super(topic, ledger, brokerService); + } + + @SneakyThrows + @Override + public CompletableFuture checkDeduplicationStatus() { + topicInitSuccessSignal.countDown(); + // Sleep 1s pending txn buffer recover failed and close topic + Thread.sleep(1000); + return CompletableFuture.completedFuture(null); + } + } + +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java index 8dabab968fb25..826fa774b8434 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java @@ -155,7 +155,7 @@ public CompletableFuture closeAsync() { } }; TransactionBufferProvider originalTransactionBufferProvider = pulsar.getTransactionBufferProvider(); - pulsar.setTransactionExecutorProvider(mockTransactionBufferProvider); + pulsar.setTransactionBufferProvider(mockTransactionBufferProvider); CompletableFuture> firstLoad = pulsar.getBrokerService().getTopic(tpName, true); Awaitility.await().ignoreExceptions().atMost(5, TimeUnit.SECONDS) .pollInterval(100, TimeUnit.MILLISECONDS) @@ -173,7 +173,7 @@ public CompletableFuture closeAsync() { } // set to back - pulsar.setTransactionExecutorProvider(originalTransactionBufferProvider); + pulsar.setTransactionBufferProvider(originalTransactionBufferProvider); pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); pulsar.getConfig().setBrokerDeduplicationEnabled(false); pulsar.getConfig().setTransactionCoordinatorEnabled(false); From 00768ba75b8821e65e9bcb9b680cef476c6ad76f Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 6 Feb 2025 19:42:09 +0800 Subject: [PATCH 02/44] Cherry-pick #23935 (cherry picked from commit df737f7e882408148e55f5836b903639f99f962b) --- .../org/apache/pulsar/client/impl/ProducerImpl.java | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) 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 50209c009c75d..6fd47e072f017 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 @@ -1836,14 +1836,6 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { cnx.sendRequestWithId(cmd, closeRequestId); } - if (cause instanceof PulsarClientException.ProducerFencedException) { - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Failed to create producer: {}", - topic, producerName, cause.getMessage()); - } - } else { - log.error("[{}] [{}] Failed to create producer: {}", topic, producerName, cause.getMessage()); - } // Close the producer since topic does not exist. if (cause instanceof PulsarClientException.TopicDoesNotExistException) { closeAsync().whenComplete((v, ex) -> { @@ -1873,6 +1865,11 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { } else if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededError) { log.warn("[{}] [{}] Producer is blocked on creation because backlog exceeded on topic.", producerName, topic); + } else if (PulsarClientException.isRetriableError(cause)) { + log.info("[{}] [{}] Temporary error in creating producer: {}", topic, + producerName, cause.getMessage()); + } else { + log.error("[{}] [{}] Failed to create producer: {}", topic, producerName, cause.getMessage()); } if (cause instanceof PulsarClientException.TopicTerminatedException) { From ce1af1007bbd9f065a659424a43479ef15f5d3d1 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Thu, 6 Feb 2025 10:07:16 +0800 Subject: [PATCH 03/44] [improve][broker] Don't print error logs for ProducerBusyException (#23929) ### Motivation When the producer's maximum count is reached, the broker will log the following error message: ``` 2025-02-05T18:31:37,996+0800 [pulsar-io-18-16] ERROR org.apache.pulsar.broker.service.ServerCnx - [/127.0.0.1:57684] Failed to create topic persistent://public/default/test2asgasgaw, producerId=1 java.util.concurrent.CompletionException: org.apache.pulsar.broker.service.BrokerServiceException$ProducerBusyException: Topic 'persistent://public/default/test2asgasgaw' reached max producers limit ``` These errors are related to the client side. The client can handle the error, so we don't need to print it in the broker log. ### Modifications - Avoid printing the error log for the ProducerBusyException (cherry picked from commit 144fe2eb634471d338a7e6ee879bc6175e5d061c) (cherry picked from commit 92982bb6d57300c27e5dac0c18f91510d355e5d2) --- .../main/java/org/apache/pulsar/broker/service/ServerCnx.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 2c09e7b2ede17..9aab88518b383 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -1657,7 +1657,8 @@ protected void handleProducer(final CommandProducer cmdProducer) { log.warn("[{}] Failed to load topic {}, producerId={}: Topic not found", remoteAddress, topicName, producerId); } else if (!Exceptions.areExceptionsPresentInChain(cause, - ServiceUnitNotReadyException.class, ManagedLedgerException.class)) { + ServiceUnitNotReadyException.class, ManagedLedgerException.class, + BrokerServiceException.ProducerBusyException.class)) { log.error("[{}] Failed to create topic {}, producerId={}", remoteAddress, topicName, producerId, exception); } From 6a2d72a959ad031ec23f53c1f2c1b335db65273f Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 14 Aug 2023 15:25:30 +0800 Subject: [PATCH 04/44] [cleanup][admin] Do not print full stacktrace when get partitioned metadata not found (#20979) (cherry picked from commit 75d4d82c5fb419c67c259f86e0342704b2ed581b) (cherry picked from commit c269e380981c5cddafcb394cdb94c21832129831) --- .../pulsar/broker/admin/v1/PersistentTopics.java | 12 ++++++++++-- .../pulsar/broker/admin/v2/PersistentTopics.java | 12 ++++++++++-- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index e9bb1a4054764..d9b7430072bd2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -41,6 +41,7 @@ import javax.ws.rs.container.Suspended; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.web.RestException; @@ -320,8 +321,15 @@ public void getPartitionedMetadata( internalGetPartitionedMetadataAsync(authoritative, checkAllowAutoCreation) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { - log.error("[{}] Failed to get partitioned metadata topic {}", clientAppId(), topicName, ex); + Throwable t = FutureUtil.unwrapCompletionException(ex); + if (!isRedirectException(t)) { + if (AdminResource.isNotFoundException(t)) { + log.error("[{}] Failed to get partitioned metadata topic {}: {}", + clientAppId(), topicName, ex.getMessage()); + } else { + log.error("[{}] Failed to get partitioned metadata topic {}", + clientAppId(), topicName, t); + } } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 1f28283737f73..39dd5640197f1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -46,6 +46,7 @@ import javax.ws.rs.core.Response; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.web.RestException; @@ -940,8 +941,15 @@ public void getPartitionedMetadata( internalGetPartitionedMetadataAsync(authoritative, checkAllowAutoCreation) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { - log.error("[{}] Failed to get partitioned metadata topic {}", clientAppId(), topicName, ex); + Throwable t = FutureUtil.unwrapCompletionException(ex); + if (!isRedirectException(t)) { + if (AdminResource.isNotFoundException(t)) { + log.error("[{}] Failed to get partitioned metadata topic {}: {}", + clientAppId(), topicName, ex.getMessage()); + } else { + log.error("[{}] Failed to get partitioned metadata topic {}", + clientAppId(), topicName, t); + } } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; From 4dd7b3ae0d2ff4c129aae0fdea86315b475bd721 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Thu, 6 Feb 2025 17:56:57 +0800 Subject: [PATCH 05/44] [improve][broker] Do not print error logs for NotFound or Conflict errors when using the Admin API (#23928) ### Motivation Currently, when there is a 404 or 409 error in the Admin API call, the broker prints the error logs. ``` ERROR org.apache.pulsar.broker.admin.v2.PersistentTopics - [xxx] Failed to get partitioned metadata topic persistent://xxx: Namespace not found ``` ``` ERROR org.apache.pulsar.broker.admin.v2.Namespaces - Failed to get policies for namespace xxx: Namespace does not exist ``` ``` ERROR org.apache.pulsar.broker.admin.v2.PersistentTopics - [xxx] Failed to create non-partitioned topic persistent:/xxx: This topic already exists ``` ``` [pulsar-web-44-1] ERROR org.apache.pulsar.broker.admin.AdminResource - [admin] Failed to create partitioned topic persistent://xxx java.util.concurrent.CompletionException: org.apache.pulsar.broker.web.RestException: This topic already exists ``` These errors are related to the client side. The client can handle the error, so we don't need to print it in the broker log. ### Modifications - Print a warning log for NotFound or Conflict errors in the Admin API. (cherry picked from commit 99dc74b0e268e549384ef2099f17c773101a79bb) (cherry picked from commit 560d120058807a61186952a29ab59365ea14423b) --- .../pulsar/broker/admin/AdminResource.java | 18 +++++++++++++++++- .../pulsar/broker/admin/v2/Namespaces.java | 7 ++++++- .../broker/admin/v2/PersistentTopics.java | 4 ++-- 3 files changed, 25 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index f1812be434438..ddd425a9952fb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -629,7 +629,12 @@ && pulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { asyncResponse.resume(Response.noContent().build()); }) .exceptionally(ex -> { - log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, ex); + if (AdminResource.isConflictException(ex)) { + log.info("[{}] Failed to create partitioned topic {}: {}", clientAppId(), topicName, + ex.getMessage()); + } else { + log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, ex); + } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); @@ -893,6 +898,10 @@ protected static boolean isRedirectException(Throwable ex) { == Status.TEMPORARY_REDIRECT.getStatusCode(); } + protected static boolean isNotFoundOrConflictException(Throwable ex) { + return isNotFoundException(ex) || isConflictException(ex); + } + protected static boolean isNotFoundException(Throwable ex) { Throwable realCause = FutureUtil.unwrapCompletionException(ex); return realCause instanceof WebApplicationException @@ -900,6 +909,13 @@ protected static boolean isNotFoundException(Throwable ex) { == Status.NOT_FOUND.getStatusCode(); } + protected static boolean isConflictException(Throwable ex) { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + return realCause instanceof WebApplicationException + && ((WebApplicationException) realCause).getResponse().getStatus() + == Status.CONFLICT.getStatusCode(); + } + protected static boolean isNot307And404Exception(Throwable ex) { return !isRedirectException(ex) && !isNotFoundException(ex); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java index f1f4c62ed3439..77f7db38ba59c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java @@ -48,6 +48,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; +import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.admin.impl.NamespacesBase; import org.apache.pulsar.broker.admin.impl.OffloaderObjectsScannerUtils; import org.apache.pulsar.broker.web.RestException; @@ -153,7 +154,11 @@ public void getPolicies(@Suspended AsyncResponse response, .thenCompose(__ -> getNamespacePoliciesAsync(namespaceName)) .thenAccept(response::resume) .exceptionally(ex -> { - log.error("Failed to get policies for namespace {}", namespaceName, ex); + if (AdminResource.isNotFoundOrConflictException(ex)) { + log.info("Failed to get policies for namespace {}: {}", namespaceName, ex.getMessage()); + } else { + log.error("Failed to get policies for namespace {}", namespaceName, ex); + } resumeAsyncResponseExceptionally(response, ex); return null; }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 39dd5640197f1..9edf039f696e3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -345,7 +345,7 @@ public void createNonPartitionedTopic( internalCreateNonPartitionedTopicAsync(authoritative, properties) .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { - if (isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex) && !isConflictException(ex)) { log.error("[{}] Failed to create non-partitioned topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -944,7 +944,7 @@ public void getPartitionedMetadata( Throwable t = FutureUtil.unwrapCompletionException(ex); if (!isRedirectException(t)) { if (AdminResource.isNotFoundException(t)) { - log.error("[{}] Failed to get partitioned metadata topic {}: {}", + log.info("[{}] Failed to get partitioned metadata topic {}: {}", clientAppId(), topicName, ex.getMessage()); } else { log.error("[{}] Failed to get partitioned metadata topic {}", From 98e508584b9ee6f98adb00937ceab4a851f9fafb Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Thu, 6 Feb 2025 21:03:10 +0800 Subject: [PATCH 06/44] [improve][broker] Avoid logging errors when there is a connection issue during subscription. (#23939) (cherry picked from commit 5e5d514174fdbc1b400df51fafaa18110f1c31a9) (cherry picked from commit 35c97fcb4db86012f4590925464c2a11f9cea07b) --- .../pulsar/broker/service/BrokerServiceException.java | 6 ++++++ .../broker/service/nonpersistent/NonPersistentTopic.java | 3 ++- .../pulsar/broker/service/persistent/PersistentTopic.java | 5 ++++- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java index 6abe40f811d1d..f54746180e5fe 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java @@ -224,6 +224,12 @@ public TopicPoliciesCacheNotInitException() { } } + public static class ConnectionClosedException extends BrokerServiceException { + public ConnectionClosedException(String msg) { + super(msg); + } + } + public static class TopicBacklogQuotaExceededException extends BrokerServiceException { @Getter private final BacklogQuota.RetentionPolicy retentionPolicy; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index ec5b20a5ab400..fdcf83185eda9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -349,7 +349,8 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St consumer.consumerName(), currentUsageCount()); } future.completeExceptionally( - new BrokerServiceException("Connection was closed while the opening the cursor ")); + new BrokerServiceException.ConnectionClosedException( + "Connection was closed while the opening the cursor ")); } else { log.info("[{}][{}] Created new subscription for {}", topic, subscriptionName, consumerId); future.complete(consumer); 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 0f6a6db2cd299..769d1c397ad8e 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 @@ -1009,7 +1009,8 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St decrementUsageCount(); return FutureUtil.failedFuture( - new BrokerServiceException("Connection was closed while the opening the cursor ")); + new BrokerServiceException.ConnectionClosedException( + "Connection was closed while the opening the cursor ")); } else { checkReplicatedSubscriptionControllerState(); if (log.isDebugEnabled()) { @@ -1046,6 +1047,8 @@ && isCompactionSubscription(subscriptionName)) { log.warn("[{}][{}] has been fenced. closing the topic {}", topic, subscriptionName, ex.getMessage()); close(); + } else if (ex.getCause() instanceof BrokerServiceException.ConnectionClosedException) { + log.warn("[{}][{}] Connection was closed while the opening the cursor", topic, subscriptionName); } else { log.error("[{}] Failed to create subscription: {}", topic, subscriptionName, ex); } From 110ab7e0061fed1e33a509e3cb0090554387dadb Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Fri, 7 Feb 2025 09:48:47 +0800 Subject: [PATCH 07/44] [improve][broker] Avoid printing log for IncompatibleSchemaException in ServerCnx (#23938) ### Motivation If the producer is created with some schema error, the broker will print many error logs like this: ``` ERROR org.apache.pulsar.broker.service.ServerCnx - Try add schema failed, remote address xxx java.util.concurrent.CompletionException: org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException: Producers cannot connect or send message without a schema to topics with a schemawhen SchemaValidationEnforced is enabled ``` This error can be reported to the client and not need to print it in the broker. ### Modifications - Avoid printing log for IncompatibleSchemaException in ServerCnx (cherry picked from commit 3c0bbee91368086189816c26357491e4fe596e01) (cherry picked from commit 0de057aa66ed1c794cdd925228577eda0e9ec21f) --- .../java/org/apache/pulsar/broker/service/ServerCnx.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 9aab88518b383..85a523df3198f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -1558,8 +1558,12 @@ protected void handleProducer(final CommandProducer cmdProducer) { BrokerServiceException.getClientErrorCode(exception), message); } - log.error("Try add schema failed, remote address {}, topic {}, producerId {}", remoteAddress, - topicName, producerId, exception); + var cause = FutureUtil.unwrapCompletionException(exception); + if (!(cause instanceof IncompatibleSchemaException)) { + log.error("Try add schema failed, remote address {}, topic {}, producerId {}", + remoteAddress, + topicName, producerId, exception); + } producers.remove(producerId, producerFuture); return null; }); From 5211328a8501e9f158d348e9fabc8143693651e7 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Sat, 8 Feb 2025 13:18:46 +0800 Subject: [PATCH 08/44] [improve][io] Allow skipping connector deployment (#23932) Signed-off-by: Zixuan Liu (cherry picked from commit e6cfd2bbcc536152bb95ce2bffdff26e34bce73f) (cherry picked from commit a5512c3eea686bfc0374acd79e4370b1b3e3ecbb) --- pom.xml | 1 + pulsar-io/common/pom.xml | 7 +++++++ pulsar-io/core/pom.xml | 7 +++++++ pulsar-io/pom.xml | 7 +++++++ 4 files changed, 22 insertions(+) diff --git a/pom.xml b/pom.xml index 5b679ab5f46ee..0300f88de528b 100644 --- a/pom.xml +++ b/pom.xml @@ -130,6 +130,7 @@ flexible messaging model and an intuitive client API. apachepulsar false false + false package package diff --git a/pulsar-io/common/pom.xml b/pulsar-io/common/pom.xml index a987526d67805..4f5610b26e67c 100644 --- a/pulsar-io/common/pom.xml +++ b/pulsar-io/common/pom.xml @@ -48,6 +48,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + false + + com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-io/core/pom.xml b/pulsar-io/core/pom.xml index 59dfd796773f9..31cf6acf733bc 100644 --- a/pulsar-io/core/pom.xml +++ b/pulsar-io/core/pom.xml @@ -40,6 +40,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + false + + com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-io/pom.xml b/pulsar-io/pom.xml index a383bf713ac7d..9a79e806cf490 100644 --- a/pulsar-io/pom.xml +++ b/pulsar-io/pom.xml @@ -148,6 +148,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.maven.plugins maven-checkstyle-plugin From 3a07bb3ac80faec95166c383818bc70907862c9e Mon Sep 17 00:00:00 2001 From: Yuri Mizushima Date: Sat, 8 Feb 2025 19:03:29 +0900 Subject: [PATCH 09/44] [feat][client] Support forward proxy for the ZTS server in pulsar-client-auth-athenz (#23947) (cherry picked from commit 54e9eb19d50b08978c953ec060226fd6c2a54775) (cherry picked from commit bf2ff389fbf477f1c1cff9202f8ae2c4e564ec1e) --- pom.xml | 2 +- .../impl/auth/AuthenticationAthenz.java | 8 ++- .../impl/auth/AuthenticationAthenzTest.java | 57 +++++++++++++++++++ 3 files changed, 64 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 0300f88de528b..9325af29350fc 100644 --- a/pom.xml +++ b/pom.xml @@ -149,7 +149,7 @@ flexible messaging model and an intuitive client API. 9.4.56.v20240826 2.5.2 2.42 - 1.10.50 + 1.10.62 0.16.0 4.5.10 7.9.2 diff --git a/pulsar-client-auth-athenz/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenz.java b/pulsar-client-auth-athenz/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenz.java index 84d81c5d94301..33f3ffb2ad500 100644 --- a/pulsar-client-auth-athenz/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenz.java +++ b/pulsar-client-auth-athenz/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenz.java @@ -63,6 +63,7 @@ public class AuthenticationAthenz implements Authentication, EncodedAuthenticati private transient KeyRefresher keyRefresher = null; private transient ZTSClient ztsClient = null; private String ztsUrl = null; + private String ztsProxyUrl = null; private String tenantDomain; private String tenantService; private String providerDomain; @@ -193,6 +194,9 @@ private void setAuthParams(Map authParams) { if (isNotBlank(authParams.get("ztsUrl"))) { this.ztsUrl = authParams.get("ztsUrl"); } + if (isNotBlank(authParams.get("ztsProxyUrl"))) { + this.ztsProxyUrl = authParams.get("ztsProxyUrl"); + } } @Override @@ -219,11 +223,11 @@ private ZTSClient getZtsClient() throws InterruptedException, IOException, KeyRe } final SSLContext sslContext = Utils.buildSSLContext(keyRefresher.getKeyManagerProxy(), keyRefresher.getTrustManagerProxy()); - ztsClient = new ZTSClient(ztsUrl, sslContext); + ztsClient = new ZTSClient(ztsUrl, ztsProxyUrl, sslContext); } else { ServiceIdentityProvider siaProvider = new SimpleServiceIdentityProvider(tenantDomain, tenantService, privateKey, keyId); - ztsClient = new ZTSClient(ztsUrl, tenantDomain, tenantService, siaProvider); + ztsClient = new ZTSClient(ztsUrl, ztsProxyUrl, tenantDomain, tenantService, siaProvider); } ztsClient.setPrefetchAutoEnable(this.autoPrefetchEnabled); } diff --git a/pulsar-client-auth-athenz/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenzTest.java b/pulsar-client-auth-athenz/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenzTest.java index b4b92eddd57f6..28261e2c977e0 100644 --- a/pulsar-client-auth-athenz/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenzTest.java +++ b/pulsar-client-auth-athenz/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenzTest.java @@ -18,10 +18,18 @@ */ package org.apache.pulsar.client.impl.auth; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import org.mockito.MockedConstruction; +import org.mockito.Mockito; import org.testng.annotations.Test; import org.apache.pulsar.common.util.ObjectMapperFactory; import static org.apache.pulsar.common.util.Codec.encode; @@ -287,4 +295,53 @@ public void testRoleHeaderSetting() throws Exception { assertEquals(auth2.getAuthData().getHttpHeaders().iterator().next().getKey(), "Test-Role-Header"); auth2.close(); } + + @Test + public void testZtsProxyUrlSetting() throws Exception { + final String ztsProxyUrl = "https://example.com:4443/"; + final String paramsStr = new String(Files.readAllBytes(Paths.get("./src/test/resources/authParams.json"))); + final ObjectMapper jsonMapper = ObjectMapperFactory.create(); + final Map authParamsMap = jsonMapper.readValue(paramsStr, new TypeReference>() { }); + + try (MockedConstruction mockedZTSClient = Mockito.mockConstruction(ZTSClient.class, (mock, context) -> { + final String actualZtsProxyUrl = (String) context.arguments().get(1); + assertNull(actualZtsProxyUrl); + + when(mock.getRoleToken(any(), any(), anyInt(), anyInt(), anyBoolean())).thenReturn(mock(RoleToken.class)); + })) { + authParamsMap.remove("ztsProxyUrl"); + final AuthenticationAthenz auth1 = new AuthenticationAthenz(); + auth1.configure(jsonMapper.writeValueAsString(authParamsMap)); + auth1.getAuthData(); + + assertEquals(mockedZTSClient.constructed().size(), 1); + + auth1.close(); + + authParamsMap.put("ztsProxyUrl", ""); + final AuthenticationAthenz auth2 = new AuthenticationAthenz(); + auth2.configure(jsonMapper.writeValueAsString(authParamsMap)); + auth2.getAuthData(); + + assertEquals(mockedZTSClient.constructed().size(), 2); + + auth2.close(); + } + + try (MockedConstruction mockedZTSClient = Mockito.mockConstruction(ZTSClient.class, (mock, context) -> { + final String actualZtsProxyUrl = (String) context.arguments().get(1); + assertEquals(actualZtsProxyUrl, ztsProxyUrl); + + when(mock.getRoleToken(any(), any(), anyInt(), anyInt(), anyBoolean())).thenReturn(mock(RoleToken.class)); + })) { + authParamsMap.put("ztsProxyUrl", ztsProxyUrl); + final AuthenticationAthenz auth3 = new AuthenticationAthenz(); + auth3.configure(jsonMapper.writeValueAsString(authParamsMap)); + auth3.getAuthData(); + + assertEquals(mockedZTSClient.constructed().size(), 1); + + auth3.close(); + } + } } From 5067801682ef0ecab7f70aa68e1bd4648f829443 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 10 Feb 2025 09:41:52 +0200 Subject: [PATCH 10/44] [fix][ci][branch-3.0] Modify existing /etc/docker/daemon.json file if it exists - In branch-3.0, -Ddocker.squash=true is used and it relies on experimental key set to true in daemon.json. The docker image build fails with "connection reset by peer" if squash is requested and the daemon doesn't support it. - The daemon.json file was overridden with #23909 changes and the experimental setting got removed. (cherry picked from commit fab143b262f556114dc5db73806e571bcf6cb159) --- .github/actions/clean-disk/action.yml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/.github/actions/clean-disk/action.yml b/.github/actions/clean-disk/action.yml index d74c3f25fc64c..fae68b86da9e8 100644 --- a/.github/actions/clean-disk/action.yml +++ b/.github/actions/clean-disk/action.yml @@ -46,6 +46,22 @@ 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 + if sudo test ! -f /etc/docker/daemon.json; then + # create daemon.json file + echo '{"data-root": "/mnt/docker"}' | sudo tee /etc/docker/daemon.json + else + # modify existing daemon.json file + sudo jq '.["data-root"]="/mnt/docker"' /etc/docker/daemon.json | sudo tee /tmp/daemon_temp.json$$ + sudo mv /tmp/daemon_temp.json$$ /etc/docker/daemon.json + fi + 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 8dec18f7f97a505c266b8ef41fae035acc097509 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 11 Feb 2025 02:17:53 +0800 Subject: [PATCH 11/44] [fix][ml] Fix memory leak due to duplicated RangeCache value retain operations (#23955) Co-authored-by: Lari Hotari (cherry picked from commit 20b3b22368b96fcfdd0aa65332b58deb4b518656) (cherry picked from commit 224320e8f4c4f2e174d9f3ce5e89c293cd418aec) --- .../bookkeeper/mledger/util/RangeCache.java | 96 +++++++------------ .../mledger/util/RangeCacheTest.java | 33 ++++++- 2 files changed, 62 insertions(+), 67 deletions(-) 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 0de6f94362215..c1de09f10a6b0 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 @@ -284,6 +284,9 @@ private Value getValueFromWrapper(Key key, EntryWrapper valueWrapper } } + /** + * @apiNote the returned value must be released if it's not null + */ private Value getValueMatchingEntry(Map.Entry> entry) { Value valueMatchingEntry = EntryWrapper.getValueMatchingMapEntry(entry); return getRetainedValueMatchingKey(entry.getKey(), valueMatchingEntry); @@ -291,6 +294,9 @@ private Value getValueMatchingEntry(Map.Entry> ent // validates that the value matches the key and that the value has not been recycled // which are possible due to the lack of exclusive locks in the cache and the use of reference counted objects + /** + * @apiNote the returned value must be released if it's not null + */ private Value getRetainedValueMatchingKey(Key key, Value value) { if (value == null) { // the wrapper has been recycled and contains another key @@ -350,7 +356,7 @@ public Pair removeRange(Key first, Key last, boolean lastInclusiv RemovalCounters counters = RemovalCounters.create(); Map> subMap = entries.subMap(first, true, last, lastInclusive); for (Map.Entry> entry : subMap.entrySet()) { - removeEntry(entry, counters, true); + removeEntry(entry, counters); } return handleRemovalResult(counters); } @@ -361,84 +367,48 @@ enum RemoveEntryResult { BREAK_LOOP; } - private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters, - boolean skipInvalid) { - return removeEntry(entry, counters, skipInvalid, x -> true); + private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters) { + return removeEntry(entry, counters, x -> true); } private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters, - boolean skipInvalid, Predicate removeCondition) { + Predicate removeCondition) { Key key = entry.getKey(); EntryWrapper entryWrapper = entry.getValue(); Value value = getValueMatchingEntry(entry); if (value == null) { - // the wrapper has already been recycled and contains another key - if (!skipInvalid) { - EntryWrapper removed = entries.remove(key); - if (removed != null) { - // log and remove the entry without releasing the value - log.info("Key {} does not match the entry's value wrapper's key {}, removed entry by key without " - + "releasing the value", key, entryWrapper.getKey()); - counters.entryRemoved(removed.getSize()); - return RemoveEntryResult.ENTRY_REMOVED; - } - } - return RemoveEntryResult.CONTINUE_LOOP; - } - try { - // add extra retain to avoid value being released while we are removing it - value.retain(); - } catch (IllegalReferenceCountException e) { - // Value was already released - if (!skipInvalid) { - // remove the specific entry without releasing the value - if (entries.remove(key, entryWrapper)) { - log.info("Value was already released for key {}, removed entry without releasing the value", key); - counters.entryRemoved(entryWrapper.getSize()); - return RemoveEntryResult.ENTRY_REMOVED; - } - } + // the wrapper has already been recycled or contains another key + entries.remove(key, entryWrapper); return RemoveEntryResult.CONTINUE_LOOP; } - if (!value.matchesKey(key)) { - // this is unexpected since the IdentityWrapper.getValue(key) already checked that the value matches the key - log.warn("Unexpected race condition. Value {} does not match the key {}. Removing entry.", value, key); - } try { if (!removeCondition.test(value)) { return RemoveEntryResult.BREAK_LOOP; } - if (!skipInvalid) { - // remove the specific entry - boolean entryRemoved = entries.remove(key, entryWrapper); - if (entryRemoved) { - counters.entryRemoved(entryWrapper.getSize()); - // check that the value hasn't been recycled in between - // there should be at least 2 references since this method adds one and the cache should have - // one reference. it is valid that the value contains references even after the key has been - // removed from the cache - if (value.refCnt() > 1) { - entryWrapper.recycle(); - // remove the cache reference - value.release(); - } else { - log.info("Unexpected refCnt {} for key {}, removed entry without releasing the value", - value.refCnt(), key); - } - } - } else if (skipInvalid && value.refCnt() > 1 && entries.remove(key, entryWrapper)) { - // when skipInvalid is true, we don't remove the entry if it doesn't match matches the key - // or the refCnt is invalid + // remove the specific entry + boolean entryRemoved = entries.remove(key, entryWrapper); + if (entryRemoved) { counters.entryRemoved(entryWrapper.getSize()); - entryWrapper.recycle(); - // remove the cache reference - value.release(); + // check that the value hasn't been recycled in between + // there should be at least 2 references since this method adds one and the cache should have + // one reference. it is valid that the value contains references even after the key has been + // removed from the cache + if (value.refCnt() > 1) { + entryWrapper.recycle(); + // remove the cache reference + value.release(); + } else { + log.info("Unexpected refCnt {} for key {}, removed entry without releasing the value", + value.refCnt(), key); + } + return RemoveEntryResult.ENTRY_REMOVED; + } else { + return RemoveEntryResult.CONTINUE_LOOP; } } finally { // remove the extra retain value.release(); } - return RemoveEntryResult.ENTRY_REMOVED; } private Pair handleRemovalResult(RemovalCounters counters) { @@ -464,7 +434,7 @@ public Pair evictLeastAccessedEntries(long minSize) { if (entry == null) { break; } - removeEntry(entry, counters, false); + removeEntry(entry, counters); } return handleRemovalResult(counters); } @@ -484,7 +454,7 @@ public Pair evictLEntriesBeforeTimestamp(long maxTimestamp) { if (entry == null) { break; } - if (removeEntry(entry, counters, false, value -> timestampExtractor.getTimestamp(value) <= maxTimestamp) + if (removeEntry(entry, counters, value -> timestampExtractor.getTimestamp(value) <= maxTimestamp) == RemoveEntryResult.BREAK_LOOP) { break; } @@ -518,7 +488,7 @@ public Pair clear() { if (entry == null) { break; } - removeEntry(entry, counters, false); + removeEntry(entry, counters); } return handleRemovalResult(counters); } 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 aa13d4b8e3488..b6914fd8efe49 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 @@ -27,13 +27,16 @@ import com.google.common.collect.Lists; import io.netty.util.AbstractReferenceCounted; import io.netty.util.ReferenceCounted; +import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import lombok.Cleanup; import lombok.Data; import org.apache.commons.lang3.tuple.Pair; import org.awaitility.Awaitility; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; public class RangeCacheTest { @@ -140,9 +143,14 @@ public void customWeighter() { assertEquals(cache.getNumberOfEntries(), 2); } + @DataProvider + public static Object[][] retainBeforeEviction() { + return new Object[][]{ { true }, { false } }; + } - @Test - public void customTimeExtraction() { + + @Test(dataProvider = "retainBeforeEviction") + public void customTimeExtraction(boolean retain) { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> x.s.length()); cache.put(1, new RefString("1")); @@ -152,13 +160,30 @@ public void customTimeExtraction() { assertEquals(cache.getSize(), 10); assertEquals(cache.getNumberOfEntries(), 4); + final var retainedEntries = cache.getRange(1, 4444); + for (final var entry : retainedEntries) { + assertEquals(entry.refCnt(), 2); + if (!retain) { + entry.release(); + } + } Pair evictedSize = cache.evictLEntriesBeforeTimestamp(3); assertEquals(evictedSize.getRight().longValue(), 6); assertEquals(evictedSize.getLeft().longValue(), 3); - assertEquals(cache.getSize(), 4); assertEquals(cache.getNumberOfEntries(), 1); + + if (retain) { + final var valueToRefCnt = retainedEntries.stream().collect(Collectors.toMap(RefString::getS, + AbstractReferenceCounted::refCnt)); + assertEquals(valueToRefCnt, Map.of("1", 1, "22", 1, "333", 1, "4444", 2)); + retainedEntries.forEach(AbstractReferenceCounted::release); + } else { + final var valueToRefCnt = retainedEntries.stream().filter(v -> v.refCnt() > 0).collect(Collectors.toMap( + RefString::getS, AbstractReferenceCounted::refCnt)); + assertEquals(valueToRefCnt, Map.of("4444", 1)); + } } @Test @@ -355,4 +380,4 @@ public void testGetKeyWithDifferentInstance() { // the value should be found assertEquals(s.s, "129"); } -} \ No newline at end of file +} From 386337f6e8b236d5ca3c7a4c67aecc8099783cab Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 10 Feb 2025 09:44:07 +0800 Subject: [PATCH 12/44] [fix] [client] call redeliver 1 msg but did 2 msgs (#23943) (cherry picked from commit 7a79c78f8e6f4b52f13be1c6441f4b007d9a00fe) (cherry picked from commit f6166c72277c7d2cb68fca0f13655a31b7ce4bdf) --- .../BatchMessageWithBatchIndexLevelTest.java | 62 +++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 32 ++++------ .../GrowableArrayBlockingQueue.java | 8 +++ 3 files changed, 82 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java index 8e902d5d1e700..52147f74f4a6e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java @@ -59,6 +59,7 @@ import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -137,6 +138,67 @@ public void testBatchMessageAck() { }); } + @DataProvider + public Object[][] enabledBatchSend() { + return new Object[][] { + {false}, + {true} + }; + } + + @Test(dataProvider = "enabledBatchSend") + @SneakyThrows + public void testBatchMessageNAck(boolean enabledBatchSend) { + final String topicName = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/tp"); + final String subscriptionName = "s1"; + ConsumerImpl consumer = (ConsumerImpl) pulsarClient.newConsumer().topic(topicName) + .subscriptionName(subscriptionName) + .receiverQueueSize(21) + .subscriptionType(SubscriptionType.Shared) + .enableBatchIndexAcknowledgment(true) + .negativeAckRedeliveryDelay(100, TimeUnit.MILLISECONDS) + .subscribe(); + Producer producer = pulsarClient.newProducer().topic(topicName) + .batchingMaxMessages(20) + .batchingMaxPublishDelay(1, TimeUnit.HOURS) + .enableBatching(enabledBatchSend) + .create(); + final PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); + final PersistentDispatcherMultipleConsumers dispatcher = + (PersistentDispatcherMultipleConsumers) topic.getSubscription(subscriptionName).getDispatcher(); + + // Send messages: 20 * 2. + for (int i = 0; i < 40; i++) { + byte[] message = ("batch-message-" + i).getBytes(); + if (i == 19 || i == 39) { + producer.newMessage().value(message).send(); + } else { + producer.newMessage().value(message).sendAsync(); + } + } + Awaitility.await().untilAsserted(() -> { + if (enabledBatchSend) { + assertEquals(consumer.numMessagesInQueue(), 40); + } else { + assertEquals(consumer.numMessagesInQueue(), 21); + } + }); + + // Negative ack and verify result/ + Message receive1 = consumer.receive(); + consumer.pause(); + consumer.negativeAcknowledge(receive1); + Awaitility.await().untilAsserted(() -> { + assertEquals(consumer.numMessagesInQueue(), 20); + assertEquals(dispatcher.getConsumers().get(0).getUnackedMessages(), 20); + }); + + // cleanup. + producer.close(); + consumer.close(); + admin.topics().delete(topicName); + } + @Test public void testBatchMessageMultiNegtiveAck() throws Exception{ final String topicName = "persistent://prop/ns-abc/batchMessageMultiNegtiveAck-" + UUID.randomUUID(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 87bef2f523b7b..9ee806da7aa91 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import static org.apache.pulsar.common.protocol.Commands.hasChecksum; import static org.apache.pulsar.common.protocol.Commands.serializeWithSize; @@ -2750,27 +2751,18 @@ private Optional createEncryptionContext(MessageMetadata msgM private int removeExpiredMessagesFromQueue(Set messageIds) { int messagesFromQueue = 0; - Message peek = incomingMessages.peek(); - if (peek != null) { - MessageIdAdv messageId = MessageIdAdvUtils.discardBatch(peek.getMessageId()); - if (!messageIds.contains(messageId)) { - // first message is not expired, then no message is expired in queue. - return 0; - } - - // try not to remove elements that are added while we remove - Message message = incomingMessages.poll(); - while (message != null) { - decreaseIncomingMessageSize(message); - messagesFromQueue++; - MessageIdAdv id = MessageIdAdvUtils.discardBatch(message.getMessageId()); - if (!messageIds.contains(id)) { - messageIds.add(id); - break; - } - message.release(); - message = incomingMessages.poll(); + Message message; + while (true) { + message = incomingMessages.pollIf(msg -> { + MessageId idPolled = MessageIdAdvUtils.discardBatch(msg.getMessageId()); + return messageIds.contains(idPolled); + }); + if (message == null) { + break; } + decreaseIncomingMessageSize(message); + messagesFromQueue++; + message.release(); } return messagesFromQueue; } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/GrowableArrayBlockingQueue.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/GrowableArrayBlockingQueue.java index 467a455ed8b3d..94bfad1fbd29b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/GrowableArrayBlockingQueue.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/GrowableArrayBlockingQueue.java @@ -32,6 +32,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.StampedLock; import java.util.function.Consumer; +import java.util.function.Predicate; import javax.annotation.Nullable; /** @@ -83,10 +84,17 @@ public T remove() { @Override public T poll() { + return pollIf(v -> true); + } + + public T pollIf(Predicate predicate) { headLock.lock(); try { if (SIZE_UPDATER.get(this) > 0) { T item = data[headIndex.value]; + if (!predicate.test(item)) { + return null; + } data[headIndex.value] = null; headIndex.value = (headIndex.value + 1) & (data.length - 1); SIZE_UPDATER.decrementAndGet(this); From 948f31dd6b45b370dd5d23ca340700d851249370 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 11 Feb 2025 11:03:39 +0800 Subject: [PATCH 13/44] [fix] [ml] incorrect non-durable cursor's backlog due to concurrently trimming ledger and non-durable cursor creation (#23951) Co-authored-by: Yunze Xu (cherry picked from commit cc7b3816ce14a2ddff18dbf04216769fdb8751db) (cherry picked from commit 16bfb1aff132386b2f349be2145de91f04951093) --- .../mledger/impl/ManagedLedgerImpl.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index e130c254a8b7f..61e4068393aab 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -1137,16 +1137,17 @@ public ManagedCursor newNonDurableCursor(Position startCursorPosition, String cu return cachedCursor; } - NonDurableCursorImpl cursor = new NonDurableCursorImpl(bookKeeper, this, cursorName, - (PositionImpl) startCursorPosition, initialPosition, isReadCompacted); - cursor.setActive(); - - log.info("[{}] Opened new cursor: {}", name, cursor); + // The backlog of a non-durable cursor could be incorrect if the cursor is created before `internalTrimLedgers` + // and added to the managed ledger after `internalTrimLedgers`. + // For more details, see https://github.com/apache/pulsar/pull/23951. synchronized (this) { + NonDurableCursorImpl cursor = new NonDurableCursorImpl(bookKeeper, this, cursorName, + (PositionImpl) startCursorPosition, initialPosition, isReadCompacted); + cursor.setActive(); + log.info("[{}] Opened new cursor: {}", name, cursor); addCursor(cursor); + return cursor; } - - return cursor; } @Override From f379b6ee85b1e04591f300519a638eefff4d363f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 11 Feb 2025 17:18:21 +0200 Subject: [PATCH 14/44] [fix][client][branch-3.0] Fix cherry-pick issue in f6166c7 (cherry picked from commit 0776b37429c418c5769cc874439f3850a71a6f66) --- .../main/java/org/apache/pulsar/client/impl/ConsumerImpl.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 9ee806da7aa91..f9d92e2bfdda3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.impl; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import static org.apache.pulsar.common.protocol.Commands.hasChecksum; import static org.apache.pulsar.common.protocol.Commands.serializeWithSize; From 76486e8a88062de2a62aee67b791e0148986e836 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 11 Feb 2025 17:14:18 +0200 Subject: [PATCH 15/44] [fix][ml] Fix deadlock in PendingReadsManager (#23958) (cherry picked from commit 367faefc2e61c68e587626d07f9d98163d3cd935) (cherry picked from commit f2aa71b79a301b326047154903e6214c69065f7a) --- .../impl/cache/PendingReadsManager.java | 71 +++++++++++-------- 1 file changed, 42 insertions(+), 29 deletions(-) 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 5944199287ee1..dd9c92e0bd295 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 @@ -211,8 +211,14 @@ private FindPendingReadOutcome findPendingRead(PendingReadKey key, ConcurrentMap private class PendingRead { final PendingReadKey key; final ConcurrentMap ledgerCache; - final List callbacks = new ArrayList<>(1); - boolean completed = false; + final List listeners = new ArrayList<>(1); + PendingReadState state = PendingReadState.INITIALISED; + + enum PendingReadState { + INITIALISED, + ATTACHED, + COMPLETED + } public PendingRead(PendingReadKey key, ConcurrentMap ledgerCache) { @@ -220,48 +226,63 @@ public PendingRead(PendingReadKey key, this.ledgerCache = ledgerCache; } - public void attach(CompletableFuture> handle) { + public synchronized void attach(CompletableFuture> handle) { + if (state != PendingReadState.INITIALISED) { + // this shouldn't ever happen. this is here to prevent misuse in future changes + throw new IllegalStateException("Unexpected state " + state + " for PendingRead for key " + key); + } + state = PendingReadState.ATTACHED; handle.whenComplete((entriesToReturn, error) -> { - // execute in the completing thread - completeAndRemoveFromCache(); + // execute in the completing thread and return a copy of the listeners + List callbacks = completeAndRemoveFromCache(); // execute the callbacks in the managed ledger executor rangeEntryCache.getManagedLedger().getExecutor().execute(() -> { if (error != null) { - readEntriesFailed(error); + readEntriesFailed(callbacks, error); } else { - readEntriesComplete(entriesToReturn); + readEntriesComplete(callbacks, entriesToReturn); } }); }); } - private synchronized void completeAndRemoveFromCache() { - completed = true; + synchronized boolean addListener(AsyncCallbacks.ReadEntriesCallback callback, + Object ctx, long startEntry, long endEntry) { + if (state == PendingReadState.COMPLETED) { + return false; + } + listeners.add(new ReadEntriesCallbackWithContext(callback, ctx, startEntry, endEntry)); + return true; + } + + private synchronized List completeAndRemoveFromCache() { + state = PendingReadState.COMPLETED; // 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); + // return a copy of the listeners + return List.copyOf(listeners); } - private synchronized void readEntriesComplete(List entriesToReturn) { + // this method isn't synchronized since that could lead to deadlocks + private void readEntriesComplete(List callbacks, + 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); + first.callback.readEntriesComplete((List) entriesToReturn, first.ctx); } else { first.callback.readEntriesComplete( - keepEntries(entriesToReturn, first.startEntry, first.endEntry), - first.ctx); + keepEntries(entriesToReturn, first.startEntry, first.endEntry), first.ctx); } } else { for (ReadEntriesCallbackWithContext callback : callbacks) { callback.callback.readEntriesComplete( - copyEntries(entriesToReturn, callback.startEntry, callback.endEntry), - callback.ctx); + copyEntries(entriesToReturn, callback.startEntry, callback.endEntry), callback.ctx); } for (EntryImpl entry : entriesToReturn) { entry.release(); @@ -269,15 +290,16 @@ private synchronized void readEntriesComplete(List entriesToReturn) { } } - private synchronized void readEntriesFailed(Throwable error) { + // this method isn't synchronized since that could lead to deadlocks + private void readEntriesFailed(List callbacks, 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)); + private static List keepEntries(List list, long startEntry, long endEntry) { + List result = new ArrayList<>((int) (endEntry - startEntry + 1)); for (EntryImpl entry : list) { long entryId = entry.getEntryId(); if (startEntry <= entryId && entryId <= endEntry) { @@ -289,7 +311,7 @@ private List keepEntries(List list, long startEntry, long endE return result; } - private List copyEntries(List entriesToReturn, long startEntry, long endEntry) { + private static List copyEntries(List entriesToReturn, long startEntry, long endEntry) { List result = new ArrayList<>((int) (endEntry - startEntry + 1)); for (EntryImpl entry : entriesToReturn) { long entryId = entry.getEntryId(); @@ -300,15 +322,6 @@ private List copyEntries(List entriesToReturn, long startEntry } return result; } - - synchronized boolean addListener(AsyncCallbacks.ReadEntriesCallback callback, - Object ctx, long startEntry, long endEntry) { - if (completed) { - return false; - } - callbacks.add(new ReadEntriesCallbackWithContext(callback, ctx, startEntry, endEntry)); - return true; - } } From a5cfb36b32778d31564f767ff51bc0703c165a01 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 11 Feb 2025 12:37:09 -0800 Subject: [PATCH 16/44] [fix][sec] Upgrade to Netty 4.1.118 (#23965) (cherry picked from commit 92527df830d992bbbaa01b6e32ff579ec6dbaac4) --- buildtools/pom.xml | 2 +- .../server/src/assemble/LICENSE.bin.txt | 54 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 52 +++++++++--------- pom.xml | 2 +- 4 files changed, 55 insertions(+), 55 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 6a9f451b5acb0..47950cfd0c772 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -47,7 +47,7 @@ 4.1 10.14.2 3.1.2 - 4.1.117.Final + 4.1.118.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 c6c1b57bd0a2d..d73e1c6862312 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -289,33 +289,33 @@ 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.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 - - io.netty-netty-tcnative-boringssl-static-2.0.69.Final-osx-aarch_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.69.Final-osx-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.69.Final-windows-x86_64.jar - - io.netty-netty-tcnative-classes-2.0.69.Final.jar + - io.netty-netty-buffer-4.1.118.Final.jar + - io.netty-netty-codec-4.1.118.Final.jar + - io.netty-netty-codec-dns-4.1.118.Final.jar + - io.netty-netty-codec-http-4.1.118.Final.jar + - io.netty-netty-codec-http2-4.1.118.Final.jar + - io.netty-netty-codec-socks-4.1.118.Final.jar + - io.netty-netty-codec-haproxy-4.1.118.Final.jar + - io.netty-netty-common-4.1.118.Final.jar + - io.netty-netty-handler-4.1.118.Final.jar + - io.netty-netty-handler-proxy-4.1.118.Final.jar + - io.netty-netty-resolver-4.1.118.Final.jar + - io.netty-netty-resolver-dns-4.1.118.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.118.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.118.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.118.Final.jar + - io.netty-netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar + - io.netty-netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.118.Final.jar + - io.netty-netty-tcnative-boringssl-static-2.0.70.Final.jar + - io.netty-netty-tcnative-boringssl-static-2.0.70.Final-linux-aarch_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.70.Final-linux-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.70.Final-osx-aarch_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.70.Final-osx-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.70.Final-windows-x86_64.jar + - io.netty-netty-tcnative-classes-2.0.70.Final.jar - io.netty.incubator-netty-incubator-transport-classes-io_uring-0.0.26.Final.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.26.Final-linux-x86_64.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.26.Final-linux-aarch_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 7bac43c0705f3..a03f0c10c468c 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -344,35 +344,35 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.26.0.jar * Netty - - 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 - - netty-tcnative-boringssl-static-2.0.69.Final-osx-aarch_64.jar - - 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-buffer-4.1.118.Final.jar + - netty-codec-4.1.118.Final.jar + - netty-codec-dns-4.1.118.Final.jar + - netty-codec-http-4.1.118.Final.jar + - netty-codec-socks-4.1.118.Final.jar + - netty-codec-haproxy-4.1.118.Final.jar + - netty-common-4.1.118.Final.jar + - netty-handler-4.1.118.Final.jar + - netty-handler-proxy-4.1.118.Final.jar + - netty-resolver-4.1.118.Final.jar + - netty-resolver-dns-4.1.118.Final.jar + - netty-transport-4.1.118.Final.jar + - netty-transport-classes-epoll-4.1.118.Final.jar + - netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.118.Final.jar + - netty-tcnative-boringssl-static-2.0.70.Final.jar + - netty-tcnative-boringssl-static-2.0.70.Final-linux-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.70.Final-linux-x86_64.jar + - netty-tcnative-boringssl-static-2.0.70.Final-osx-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.70.Final-osx-x86_64.jar + - netty-tcnative-boringssl-static-2.0.70.Final-windows-x86_64.jar + - netty-tcnative-classes-2.0.70.Final.jar - 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.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-resolver-dns-classes-macos-4.1.118.Final.jar + - netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.118.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 9325af29350fc..ea3225d532e7e 100644 --- a/pom.xml +++ b/pom.xml @@ -144,7 +144,7 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.7.1 - 4.1.117.Final + 4.1.118.Final 0.0.26.Final 9.4.56.v20240826 2.5.2 From fc215d7cdf45972fb82f3be73ee4ffde38e612a8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 11 Feb 2025 23:39:13 +0200 Subject: [PATCH 17/44] [fix][ml] Fix memory leaks in ManagedCursorInfo and ManagedLedgerInfo decompression and compression (#23960) (cherry picked from commit 7f595cd9dc7522dd597ff58aa8b2a23737c9d841) (cherry picked from commit 77dc34db885d1bca261351afc1b7c95cdc8bba1b) --- .../mledger/impl/MetaStoreImpl.java | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java index e47443e4e8f95..611d9d60202cd 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java @@ -456,8 +456,13 @@ public ManagedLedgerInfo parseManagedLedgerInfo(byte[] data) throws InvalidProto try { MLDataFormats.ManagedLedgerInfoMetadata metadata = MLDataFormats.ManagedLedgerInfoMetadata.parseFrom(metadataBytes); - return ManagedLedgerInfo.parseFrom(getCompressionCodec(metadata.getCompressionType()) - .decode(byteBuf, metadata.getUncompressedSize()).nioBuffer()); + ByteBuf uncompressed = getCompressionCodec(metadata.getCompressionType()) + .decode(byteBuf, metadata.getUncompressedSize()); + try { + return ManagedLedgerInfo.parseFrom(uncompressed.nioBuffer()); + } finally { + uncompressed.release(); + } } catch (Exception e) { log.error("Failed to parse managedLedgerInfo metadata, " + "fall back to parse managedLedgerInfo directly.", e); @@ -478,8 +483,13 @@ public ManagedCursorInfo parseManagedCursorInfo(byte[] data) throws InvalidProto try { MLDataFormats.ManagedCursorInfoMetadata metadata = MLDataFormats.ManagedCursorInfoMetadata.parseFrom(metadataBytes); - return ManagedCursorInfo.parseFrom(getCompressionCodec(metadata.getCompressionType()) - .decode(byteBuf, metadata.getUncompressedSize()).nioBuffer()); + ByteBuf uncompressed = getCompressionCodec(metadata.getCompressionType()) + .decode(byteBuf, metadata.getUncompressedSize()); + try { + return ManagedCursorInfo.parseFrom(uncompressed.nioBuffer()); + } finally { + uncompressed.release(); + } } catch (Exception e) { log.error("Failed to parse ManagedCursorInfo metadata, " + "fall back to parse ManagedCursorInfo directly", e); @@ -503,29 +513,23 @@ private byte[] compressManagedInfo(byte[] info, byte[] metadata, int metadataSer if (compressionType == null || compressionType.equals(CompressionType.NONE)) { return info; } - ByteBuf metadataByteBuf = null; - ByteBuf encodeByteBuf = null; + + CompositeByteBuf compositeByteBuf = PulsarByteBufAllocator.DEFAULT.compositeBuffer(); try { - metadataByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(metadataSerializedSize + 6, + ByteBuf metadataByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(metadataSerializedSize + 6, metadataSerializedSize + 6); metadataByteBuf.writeShort(MAGIC_MANAGED_INFO_METADATA); metadataByteBuf.writeInt(metadataSerializedSize); metadataByteBuf.writeBytes(metadata); - encodeByteBuf = getCompressionCodec(compressionType) + ByteBuf encodeByteBuf = getCompressionCodec(compressionType) .encode(Unpooled.wrappedBuffer(info)); - CompositeByteBuf compositeByteBuf = PulsarByteBufAllocator.DEFAULT.compositeBuffer(); compositeByteBuf.addComponent(true, metadataByteBuf); compositeByteBuf.addComponent(true, encodeByteBuf); byte[] dataBytes = new byte[compositeByteBuf.readableBytes()]; compositeByteBuf.readBytes(dataBytes); return dataBytes; } finally { - if (metadataByteBuf != null) { - metadataByteBuf.release(); - } - if (encodeByteBuf != null) { - encodeByteBuf.release(); - } + compositeByteBuf.release(); } } From 12d28c0e78c0c09803f1cdba8d9898d3d0896104 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 12 Feb 2025 07:30:02 +0800 Subject: [PATCH 18/44] [improve][broker] Avoid PersistentReplicator.expireMessages logic compute backlog twice (#23957) (cherry picked from commit 6db275c3407ef7b7d308ac2eeaa3581f0d3576be) (cherry picked from commit 921b75e23c82c42652e6bc908dc2bc1fc27adacb) --- .../broker/service/persistent/PersistentReplicator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 4d74aa83c40d0..a53f56d4abce1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -632,8 +632,8 @@ private long getReplicationDelayInSeconds() { @Override public boolean expireMessages(int messageTTLInSeconds) { - if ((cursor.getNumberOfEntriesInBacklog(false) == 0) - || (cursor.getNumberOfEntriesInBacklog(false) < MINIMUM_BACKLOG_FOR_EXPIRY_CHECK + long backlog = cursor.getNumberOfEntriesInBacklog(false); + if ((backlog == 0) || (backlog < MINIMUM_BACKLOG_FOR_EXPIRY_CHECK && !topic.isOldestMessageExpired(cursor, messageTTLInSeconds))) { // don't do anything for almost caught-up connected subscriptions return false; From 06a08aacc6c468f4ad0c7eb94933d4c6322d4f2f Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 12 Feb 2025 23:06:21 +0800 Subject: [PATCH 19/44] [fix][build] Upgrade json-smart to 2.5.2 (#23966) Signed-off-by: Zixuan Liu (cherry picked from commit 3cc22063c0077afacaf850ae1069fa51a82d327a) (cherry picked from commit 2557d9d9be1dcc59a98edf68f359eca3afe036f8) --- pom.xml | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index ea3225d532e7e..a6fbf06179591 100644 --- a/pom.xml +++ b/pom.xml @@ -194,8 +194,8 @@ flexible messaging model and an intuitive client API. 2.7.5 0.4.4-hotfix1 3.3.5 - 2.4.10 - 2.16.0 + 2.5.2 + 1.2.4 8.12.1 368 2.6.1.Final @@ -316,6 +316,11 @@ flexible messaging model and an intuitive client API. + + net.minidev + json-smart + ${json-smart.version} + org.jline From 1464accaf9e46d351083f2d7145a03660dec0ce6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 12 Feb 2025 21:57:08 +0200 Subject: [PATCH 20/44] [improve][ci] Skip "OWASP dependency check" when data wasn't found in cache (#23970) (cherry picked from commit 0a9597640ff922c00b49016af839d7c56086f82b) (cherry picked from commit d717da1e8363d52b0717e93981ac39a506647345) --- .github/workflows/pulsar-ci.yaml | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 9c1986c5b5b43..ee0fb27307708 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -1312,15 +1312,21 @@ jobs: restore-keys: | owasp-dependency-check-data- + - name: Log warning when skipped + if: ${{ !steps.restore-owasp-dependency-check-data.outputs.cache-matched-key }} + run: | + echo "::warning::OWASP Dependency Check was skipped since the OWASP Dependency check data wasn't found in the cache. Run ci-owasp-dependency-check.yaml workflow to update the cache." + # Projects dependent on flume, hdfs, hbase, and presto currently excluded from the scan. - name: trigger dependency check + if: ${{ steps.restore-owasp-dependency-check-data.outputs.cache-matched-key }} run: | mvn -B -ntp verify -PskipDocker,skip-all,owasp-dependency-check -Dcheckstyle.skip=true -DskipTests \ -pl '!pulsar-sql,!distribution/server,!distribution/io,!distribution/offloaders,!pulsar-sql/presto-distribution,!tiered-storage/file-system,!pulsar-io/flume,!pulsar-io/hbase,!pulsar-io/hdfs2,!pulsar-io/hdfs3,!pulsar-io/docs,!pulsar-io/jdbc/openmldb' - name: Upload report uses: actions/upload-artifact@v4 - if: ${{ cancelled() || failure() }} + if: ${{ steps.restore-owasp-dependency-check-data.outputs.cache-matched-key && (cancelled() || failure()) }} continue-on-error: true with: name: dependency report From 5e478433c342a957ea630eac6f5a4e522bc14ad1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Feb 2025 14:38:45 -0800 Subject: [PATCH 21/44] [fix] Bump org.apache.solr:solr-core from 8.11.3 to 9.8.0 in /pulsar-io/solr (#23899) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli (cherry picked from commit 014f0a84638b4da990a3f805ef0192b66e983344) --- pulsar-io/solr/pom.xml | 25 ++++++++++++++++++- .../apache/pulsar/io/solr/SolrServerUtil.java | 2 +- 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 02972032b81b9..f2f45572d5918 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -29,12 +29,29 @@ - 8.11.3 + 9.8.0 pulsar-io-solr Pulsar IO :: Solr + + + + org.eclipse.jetty + jetty-bom + 10.0.22 + pom + import + + + org.eclipse.jetty + jetty-server + 10.0.22 + + + + ${project.groupId} @@ -61,6 +78,12 @@ solr-solrj ${solr.version} + + org.apache.solr + solr-test-framework + ${solr.version} + test + org.apache.solr solr-core diff --git a/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrServerUtil.java b/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrServerUtil.java index cc59c6dce6bf6..270abc8da2439 100644 --- a/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrServerUtil.java +++ b/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrServerUtil.java @@ -20,7 +20,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.FileUtils; -import org.apache.solr.client.solrj.embedded.JettySolrRunner; +import org.apache.solr.embedded.JettySolrRunner; import java.io.File; From 64b2eb19f9b59441d1a92c55c36fd357ac1f6a79 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 13 Feb 2025 01:29:54 +0200 Subject: [PATCH 22/44] [fix][client] Fix memory leak in ClientCnx.newLookup when there's TooManyRequestsException (#23971) (cherry picked from commit 9d767b3d81da7d9d8bdb4d72098f0b1243c9c9f7) --- .../src/main/java/org/apache/pulsar/client/impl/ClientCnx.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index b8caa0f438361..675fa2063473f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -859,6 +859,7 @@ public CompletableFuture newLookup(ByteBuf request, long reque if (maxLookupRequestSemaphore.tryAcquire()) { waitingLookupRequests.add(Pair.of(requestId, Pair.of(request, future))); } else { + request.release(); if (log.isDebugEnabled()) { log.debug("{} Failed to add lookup-request into waiting queue", requestId); } From 1cd1f1aa45b73bc63bd1f109dd99475787004353 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 13 Feb 2025 01:33:55 +0200 Subject: [PATCH 23/44] [fix][client] Fix memory leak when message size exceeds max message size and batching is enabled (#23967) (cherry picked from commit 2620871584ef7c76f8f90fc6c8a341ecc52ecb61) --- .../apache/pulsar/client/impl/BatchMessageContainerImpl.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java index a3cd84981fbb1..e112e1b866c32 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java @@ -273,6 +273,7 @@ public OpSendMsg createOpSendMsg() throws IOException { // handle mgs size check as non-batched in `ProducerImpl.isMessageSizeExceeded` if (op.getMessageHeaderAndPayloadSize() > getMaxMessageSize()) { + cmd.release(); producer.semaphoreRelease(1); producer.client.getMemoryLimitController().releaseMemory( messages.get(0).getUncompressedSize() + batchAllocatedSizeBytes); @@ -286,6 +287,7 @@ public OpSendMsg createOpSendMsg() throws IOException { ByteBuf encryptedPayload = producer.encryptMessage(messageMetadata, getCompressedBatchMetadataAndPayload()); updateAndReserveBatchAllocatedSize(encryptedPayload.capacity()); if (encryptedPayload.readableBytes() > getMaxMessageSize()) { + encryptedPayload.release(); producer.semaphoreRelease(messages.size()); messages.forEach(msg -> producer.client.getMemoryLimitController() .releaseMemory(msg.getUncompressedSize())); From 78e76ddeb820495c02ddeb6523639148ea46456f Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 14 Feb 2025 01:50:53 +0800 Subject: [PATCH 24/44] [fix][io] Fix pulsar-io:pom not found (#23979) Signed-off-by: Zixuan Liu (cherry picked from commit d3ea0ee8515949808f2067c3cc2874ac379b5f28) (cherry picked from commit 10ae34c607accf72ede379c10b6ecb284f636b3f) --- pulsar-io/aerospike/pom.xml | 7 +++++++ pulsar-io/alluxio/pom.xml | 7 +++++++ pulsar-io/aws/pom.xml | 7 +++++++ pulsar-io/batch-data-generator/pom.xml | 7 +++++++ pulsar-io/canal/pom.xml | 7 +++++++ pulsar-io/cassandra/pom.xml | 7 +++++++ pulsar-io/common/pom.xml | 7 ------- pulsar-io/core/pom.xml | 7 ------- pulsar-io/data-generator/pom.xml | 7 +++++++ pulsar-io/debezium/core/pom.xml | 12 ++++++++++++ pulsar-io/debezium/mongodb/pom.xml | 7 +++++++ pulsar-io/debezium/mssql/pom.xml | 7 +++++++ pulsar-io/debezium/mysql/pom.xml | 7 +++++++ pulsar-io/debezium/oracle/pom.xml | 7 +++++++ pulsar-io/debezium/postgres/pom.xml | 7 +++++++ pulsar-io/docs/pom.xml | 7 +++++++ pulsar-io/dynamodb/pom.xml | 7 +++++++ pulsar-io/elastic-search/pom.xml | 7 +++++++ pulsar-io/file/pom.xml | 7 +++++++ pulsar-io/flume/pom.xml | 7 +++++++ pulsar-io/hbase/pom.xml | 7 +++++++ pulsar-io/hdfs3/pom.xml | 7 +++++++ pulsar-io/http/pom.xml | 7 +++++++ pulsar-io/influxdb/pom.xml | 7 +++++++ pulsar-io/jdbc/clickhouse/pom.xml | 7 +++++++ pulsar-io/jdbc/core/pom.xml | 12 ++++++++++++ pulsar-io/jdbc/mariadb/pom.xml | 7 +++++++ pulsar-io/jdbc/openmldb/pom.xml | 7 +++++++ pulsar-io/jdbc/postgres/pom.xml | 7 +++++++ pulsar-io/jdbc/sqlite/pom.xml | 7 +++++++ pulsar-io/kafka-connect-adaptor-nar/pom.xml | 7 +++++++ pulsar-io/kafka-connect-adaptor/pom.xml | 11 +++++++++++ pulsar-io/kafka/pom.xml | 7 +++++++ pulsar-io/kinesis/pom.xml | 7 +++++++ pulsar-io/mongo/pom.xml | 7 +++++++ pulsar-io/netty/pom.xml | 7 +++++++ pulsar-io/nsq/pom.xml | 7 +++++++ pulsar-io/pom.xml | 7 ------- pulsar-io/rabbitmq/pom.xml | 7 +++++++ pulsar-io/redis/pom.xml | 7 +++++++ pulsar-io/solr/pom.xml | 7 +++++++ pulsar-io/twitter/pom.xml | 7 +++++++ 42 files changed, 287 insertions(+), 21 deletions(-) diff --git a/pulsar-io/aerospike/pom.xml b/pulsar-io/aerospike/pom.xml index d2ff351f63e0b..6d8c832bb5b5d 100644 --- a/pulsar-io/aerospike/pom.xml +++ b/pulsar-io/aerospike/pom.xml @@ -68,6 +68,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/alluxio/pom.xml b/pulsar-io/alluxio/pom.xml index 0b3415bb41436..6c1a5a9c65fb8 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -109,6 +109,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/aws/pom.xml b/pulsar-io/aws/pom.xml index 050f99e662367..0afe01b2db786 100644 --- a/pulsar-io/aws/pom.xml +++ b/pulsar-io/aws/pom.xml @@ -59,6 +59,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-io/batch-data-generator/pom.xml b/pulsar-io/batch-data-generator/pom.xml index aad0318b1f2f7..44c7207e0065f 100644 --- a/pulsar-io/batch-data-generator/pom.xml +++ b/pulsar-io/batch-data-generator/pom.xml @@ -73,6 +73,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/canal/pom.xml b/pulsar-io/canal/pom.xml index 6230a6f7121c9..f557c495aab27 100644 --- a/pulsar-io/canal/pom.xml +++ b/pulsar-io/canal/pom.xml @@ -123,6 +123,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/cassandra/pom.xml b/pulsar-io/cassandra/pom.xml index f506df6861662..14154e0b31430 100644 --- a/pulsar-io/cassandra/pom.xml +++ b/pulsar-io/cassandra/pom.xml @@ -57,6 +57,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/common/pom.xml b/pulsar-io/common/pom.xml index 4f5610b26e67c..a987526d67805 100644 --- a/pulsar-io/common/pom.xml +++ b/pulsar-io/common/pom.xml @@ -48,13 +48,6 @@ - - org.apache.maven.plugins - maven-deploy-plugin - - false - - com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-io/core/pom.xml b/pulsar-io/core/pom.xml index 31cf6acf733bc..59dfd796773f9 100644 --- a/pulsar-io/core/pom.xml +++ b/pulsar-io/core/pom.xml @@ -40,13 +40,6 @@ - - org.apache.maven.plugins - maven-deploy-plugin - - false - - com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-io/data-generator/pom.xml b/pulsar-io/data-generator/pom.xml index 28a3e4725db5b..3b7a9a0a9c295 100644 --- a/pulsar-io/data-generator/pom.xml +++ b/pulsar-io/data-generator/pom.xml @@ -60,6 +60,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/debezium/core/pom.xml b/pulsar-io/debezium/core/pom.xml index 1ce97c8c92d7e..50f65c8e5df14 100644 --- a/pulsar-io/debezium/core/pom.xml +++ b/pulsar-io/debezium/core/pom.xml @@ -116,4 +116,16 @@ + + + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + + + + diff --git a/pulsar-io/debezium/mongodb/pom.xml b/pulsar-io/debezium/mongodb/pom.xml index d6c44f98f622e..e53b3ac0e4b69 100644 --- a/pulsar-io/debezium/mongodb/pom.xml +++ b/pulsar-io/debezium/mongodb/pom.xml @@ -55,6 +55,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/debezium/mssql/pom.xml b/pulsar-io/debezium/mssql/pom.xml index ae227e115db3f..0299dfb2e8174 100644 --- a/pulsar-io/debezium/mssql/pom.xml +++ b/pulsar-io/debezium/mssql/pom.xml @@ -55,6 +55,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/debezium/mysql/pom.xml b/pulsar-io/debezium/mysql/pom.xml index d303aa514eabc..f8cc07a57cd0f 100644 --- a/pulsar-io/debezium/mysql/pom.xml +++ b/pulsar-io/debezium/mysql/pom.xml @@ -65,6 +65,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/debezium/oracle/pom.xml b/pulsar-io/debezium/oracle/pom.xml index b53a837197c02..98b2b96a5ebb4 100644 --- a/pulsar-io/debezium/oracle/pom.xml +++ b/pulsar-io/debezium/oracle/pom.xml @@ -55,6 +55,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/debezium/postgres/pom.xml b/pulsar-io/debezium/postgres/pom.xml index bdccad78bfddf..e33d45476d35e 100644 --- a/pulsar-io/debezium/postgres/pom.xml +++ b/pulsar-io/debezium/postgres/pom.xml @@ -62,6 +62,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/docs/pom.xml b/pulsar-io/docs/pom.xml index a98f6cfdd80a7..2149def2eb51b 100644 --- a/pulsar-io/docs/pom.xml +++ b/pulsar-io/docs/pom.xml @@ -221,6 +221,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-io/dynamodb/pom.xml b/pulsar-io/dynamodb/pom.xml index e3e1285d1ec15..7347e8b98dcda 100644 --- a/pulsar-io/dynamodb/pom.xml +++ b/pulsar-io/dynamodb/pom.xml @@ -101,6 +101,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/elastic-search/pom.xml b/pulsar-io/elastic-search/pom.xml index 8a303fc781e70..9302b811597e4 100644 --- a/pulsar-io/elastic-search/pom.xml +++ b/pulsar-io/elastic-search/pom.xml @@ -113,6 +113,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/file/pom.xml b/pulsar-io/file/pom.xml index 1501e0c93257b..19f79cf8c6d07 100644 --- a/pulsar-io/file/pom.xml +++ b/pulsar-io/file/pom.xml @@ -60,6 +60,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/flume/pom.xml b/pulsar-io/flume/pom.xml index c63eed31c2e6d..5f3bfd41567a4 100644 --- a/pulsar-io/flume/pom.xml +++ b/pulsar-io/flume/pom.xml @@ -126,6 +126,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/hbase/pom.xml b/pulsar-io/hbase/pom.xml index 9ff47bfa572cb..9f45d33048cc8 100644 --- a/pulsar-io/hbase/pom.xml +++ b/pulsar-io/hbase/pom.xml @@ -89,6 +89,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index 2d72dc2a91ef0..558fc02fd4521 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -83,6 +83,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/http/pom.xml b/pulsar-io/http/pom.xml index 501103aa16f8f..c6a032d453f55 100644 --- a/pulsar-io/http/pom.xml +++ b/pulsar-io/http/pom.xml @@ -77,6 +77,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/influxdb/pom.xml b/pulsar-io/influxdb/pom.xml index 139c65e4e1e65..25356d3e38543 100644 --- a/pulsar-io/influxdb/pom.xml +++ b/pulsar-io/influxdb/pom.xml @@ -83,6 +83,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/jdbc/clickhouse/pom.xml b/pulsar-io/jdbc/clickhouse/pom.xml index 36a20297bd4f4..c6c77abb8e74a 100644 --- a/pulsar-io/jdbc/clickhouse/pom.xml +++ b/pulsar-io/jdbc/clickhouse/pom.xml @@ -54,6 +54,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index 84f324b56a2ae..d383b4d5576b1 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -80,4 +80,16 @@ + + + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + + + + \ No newline at end of file diff --git a/pulsar-io/jdbc/mariadb/pom.xml b/pulsar-io/jdbc/mariadb/pom.xml index 236325d01b852..91cba644c7add 100644 --- a/pulsar-io/jdbc/mariadb/pom.xml +++ b/pulsar-io/jdbc/mariadb/pom.xml @@ -46,6 +46,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/jdbc/openmldb/pom.xml b/pulsar-io/jdbc/openmldb/pom.xml index 78b3f060a194b..64f49edf04262 100644 --- a/pulsar-io/jdbc/openmldb/pom.xml +++ b/pulsar-io/jdbc/openmldb/pom.xml @@ -63,6 +63,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/jdbc/postgres/pom.xml b/pulsar-io/jdbc/postgres/pom.xml index 28673373582cd..375268333c1e0 100644 --- a/pulsar-io/jdbc/postgres/pom.xml +++ b/pulsar-io/jdbc/postgres/pom.xml @@ -47,6 +47,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/jdbc/sqlite/pom.xml b/pulsar-io/jdbc/sqlite/pom.xml index 42d878d44e1c6..238c4dc7912b8 100644 --- a/pulsar-io/jdbc/sqlite/pom.xml +++ b/pulsar-io/jdbc/sqlite/pom.xml @@ -63,6 +63,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/kafka-connect-adaptor-nar/pom.xml b/pulsar-io/kafka-connect-adaptor-nar/pom.xml index 6d690b100186c..b4ae4a79ee999 100644 --- a/pulsar-io/kafka-connect-adaptor-nar/pom.xml +++ b/pulsar-io/kafka-connect-adaptor-nar/pom.xml @@ -41,6 +41,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index b7608b6f24e81..684c4aab3b2a7 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -208,4 +208,15 @@ + + + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + + + diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index 328bd2f061a17..d821f022bb283 100644 --- a/pulsar-io/kafka/pom.xml +++ b/pulsar-io/kafka/pom.xml @@ -130,6 +130,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/kinesis/pom.xml b/pulsar-io/kinesis/pom.xml index 0034f9c4cad5c..be2ecf7826fe2 100644 --- a/pulsar-io/kinesis/pom.xml +++ b/pulsar-io/kinesis/pom.xml @@ -149,6 +149,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/mongo/pom.xml b/pulsar-io/mongo/pom.xml index 4de5e13767ec4..66449e1efb932 100644 --- a/pulsar-io/mongo/pom.xml +++ b/pulsar-io/mongo/pom.xml @@ -72,6 +72,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/netty/pom.xml b/pulsar-io/netty/pom.xml index b0c7807c58f1c..82b3e76d39c20 100644 --- a/pulsar-io/netty/pom.xml +++ b/pulsar-io/netty/pom.xml @@ -72,6 +72,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/nsq/pom.xml b/pulsar-io/nsq/pom.xml index 3ac1b0cb604e6..5933343a302b9 100644 --- a/pulsar-io/nsq/pom.xml +++ b/pulsar-io/nsq/pom.xml @@ -68,6 +68,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/pom.xml b/pulsar-io/pom.xml index 9a79e806cf490..a383bf713ac7d 100644 --- a/pulsar-io/pom.xml +++ b/pulsar-io/pom.xml @@ -148,13 +148,6 @@ - - org.apache.maven.plugins - maven-deploy-plugin - - ${skipDeployConnector} - - org.apache.maven.plugins maven-checkstyle-plugin diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index 18e5fec8be407..87fab479d9cec 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -96,6 +96,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/redis/pom.xml b/pulsar-io/redis/pom.xml index 19e00c052860e..96b0c0aad4ee1 100644 --- a/pulsar-io/redis/pom.xml +++ b/pulsar-io/redis/pom.xml @@ -88,6 +88,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index f2f45572d5918..138131cb793d3 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -108,6 +108,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin diff --git a/pulsar-io/twitter/pom.xml b/pulsar-io/twitter/pom.xml index ee6e87f07060f..cbea3b90cc99c 100644 --- a/pulsar-io/twitter/pom.xml +++ b/pulsar-io/twitter/pom.xml @@ -74,6 +74,13 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + org.apache.nifi nifi-nar-maven-plugin From 61c09c94f413c3aee7e9a9d3f5cf99168e78b53e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 08:56:40 +0200 Subject: [PATCH 25/44] [improve][proxy] Make keep-alive interval configurable in Pulsar Proxy (#23981) (cherry picked from commit eb1391a199d2c85f1ad7ce22a4d3eefa78d2fed3) (cherry picked from commit ebcbfffe06ecb0c8dc16785b2b9f0b521b2b9023) --- conf/proxy.conf | 7 + .../proxy/server/ProxyConfiguration.java | 9 + .../pulsar/proxy/server/ProxyConnection.java | 2 +- .../proxy/server/ProxyConfigurationTest.java | 16 +- .../server/ProxyOriginalClientIPTest.java | 3 +- ...roxyServiceStarterDisableZeroCopyTest.java | 2 +- .../proxy/server/ProxyServiceStarterTest.java | 45 +++- .../server/ProxyServiceTlsStarterTest.java | 22 +- pulsar-proxy/src/test/resources/proxy.conf | 250 ------------------ 9 files changed, 76 insertions(+), 280 deletions(-) delete mode 100644 pulsar-proxy/src/test/resources/proxy.conf diff --git a/conf/proxy.conf b/conf/proxy.conf index 6e6c960e8009e..cda1034d65392 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -59,6 +59,13 @@ bindAddress=0.0.0.0 # If not set, the value of `InetAddress.getLocalHost().getCanonicalHostName()` is used. advertisedAddress= +# Specifies the interval (in seconds) for sending ping messages to the client. Set to 0 to disable +# ping messages. This setting applies to client connections used for topic lookups and +# partition metadata requests. When a client establishes a broker connection via the proxy, +# the client and broker will communicate directly without the proxy intercepting the messages. +# In that case, the broker's keepAliveIntervalSeconds configuration becomes relevant. +keepAliveIntervalSeconds=30 + # Enable or disable the HAProxy protocol. # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index 3d4c9e19aa4b0..1b6f213a62216 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java @@ -270,6 +270,15 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private String advertisedAddress; + @FieldContext( + category = CATEGORY_SERVER, + doc = "Specifies the interval (in seconds) for sending ping messages to the client. Set to 0 to disable " + + "ping messages. This setting applies to client connections used for topic lookups and " + + "partition metadata requests. When a client establishes a broker connection via the proxy, " + + "the client and broker will communicate directly without the proxy intercepting the messages. " + + "In that case, the broker's keepAliveIntervalSeconds configuration becomes relevant.") + private int keepAliveIntervalSeconds = 30; + @FieldContext(category = CATEGORY_SERVER, doc = "Enable or disable the proxy protocol.") private boolean haProxyProtocolEnabled; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java index 64c04c3f217ed..ec7e8ff26513e 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java @@ -155,7 +155,7 @@ ConnectionPool getConnectionPool() { } public ProxyConnection(ProxyService proxyService, DnsAddressResolverGroup dnsAddressResolverGroup) { - super(30, TimeUnit.SECONDS); + super(proxyService.getConfiguration().getKeepAliveIntervalSeconds(), TimeUnit.SECONDS); this.service = proxyService; this.dnsAddressResolverGroup = dnsAddressResolverGroup; this.state = State.Init; diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConfigurationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConfigurationTest.java index a9a562e04c899..18e7efbd7b5c6 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConfigurationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConfigurationTest.java @@ -147,7 +147,7 @@ public void testBrokerUrlCheck() throws IOException { theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) .thenReturn(configuration); try { - new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + new ProxyServiceStarter(ProxyServiceStarterTest.getArgs()); fail("brokerServiceURL must start with pulsar://"); } catch (Exception ex) { assertTrue(ex.getMessage().contains("brokerServiceURL must start with pulsar://")); @@ -161,7 +161,7 @@ public void testBrokerUrlCheck() throws IOException { theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) .thenReturn(configuration); try { - new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + new ProxyServiceStarter(ProxyServiceStarterTest.getArgs()); fail("brokerServiceURLTLS must start with pulsar+ssl://"); } catch (Exception ex) { assertTrue(ex.getMessage().contains("brokerServiceURLTLS must start with pulsar+ssl://")); @@ -174,7 +174,7 @@ public void testBrokerUrlCheck() throws IOException { theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) .thenReturn(configuration); try { - new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + new ProxyServiceStarter(ProxyServiceStarterTest.getArgs()); fail("brokerServiceURL does not support multi urls yet"); } catch (Exception ex) { assertTrue(ex.getMessage().contains("does not support multi urls yet")); @@ -188,7 +188,7 @@ public void testBrokerUrlCheck() throws IOException { theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) .thenReturn(configuration); try { - new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + new ProxyServiceStarter(ProxyServiceStarterTest.getArgs()); fail("brokerServiceURLTLS does not support multi urls yet"); } catch (Exception ex) { assertTrue(ex.getMessage().contains("does not support multi urls yet")); @@ -202,7 +202,7 @@ public void testBrokerUrlCheck() throws IOException { theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) .thenReturn(configuration); try { - new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + new ProxyServiceStarter(ProxyServiceStarterTest.getArgs()); fail("brokerWebServiceURL does not support multi urls yet"); } catch (Exception ex) { assertTrue(ex.getMessage().contains("does not support multi urls yet")); @@ -216,7 +216,7 @@ public void testBrokerUrlCheck() throws IOException { theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) .thenReturn(configuration); try { - new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + new ProxyServiceStarter(ProxyServiceStarterTest.getArgs()); fail("brokerWebServiceURLTLS does not support multi urls yet"); } catch (Exception ex) { assertTrue(ex.getMessage().contains("does not support multi urls yet")); @@ -230,7 +230,7 @@ public void testBrokerUrlCheck() throws IOException { theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) .thenReturn(configuration); try { - new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + new ProxyServiceStarter(ProxyServiceStarterTest.getArgs()); fail("functionWorkerWebServiceURL does not support multi urls yet"); } catch (Exception ex) { assertTrue(ex.getMessage().contains("does not support multi urls yet")); @@ -244,7 +244,7 @@ public void testBrokerUrlCheck() throws IOException { theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) .thenReturn(configuration); try { - new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + new ProxyServiceStarter(ProxyServiceStarterTest.getArgs()); fail("functionWorkerWebServiceURLTLS does not support multi urls yet"); } catch (Exception ex) { assertTrue(ex.getMessage().contains("does not support multi urls yet")); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java index b267439d47113..50ae6e627e820 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java @@ -39,7 +39,6 @@ @Slf4j @Test(groups = "broker") public class ProxyOriginalClientIPTest extends MockedPulsarServiceBaseTest { - static final String[] ARGS = new String[]{"-c", "./src/test/resources/proxy.conf"}; HttpClient httpClient; ProxyServiceStarter serviceStarter; String webServiceUrl; @@ -49,7 +48,7 @@ public class ProxyOriginalClientIPTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS, proxyConfig -> { + serviceStarter = new ProxyServiceStarter(ProxyServiceStarterTest.getArgs(), proxyConfig -> { proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); proxyConfig.setBrokerWebServiceURL(pulsar.getWebServiceAddress()); proxyConfig.setWebServicePort(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java index 9eaa992741635..ae20f6b85cacf 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java @@ -27,7 +27,7 @@ public class ProxyServiceStarterDisableZeroCopyTest extends ProxyServiceStarterT @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS, null, true); + serviceStarter = new ProxyServiceStarter(getArgs(), null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); serviceStarter.getConfig().setWebServicePort(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java index e12415b0d91b4..e40c37aaa06c5 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java @@ -21,16 +21,23 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.net.URI; import java.nio.ByteBuffer; import java.util.Base64; import java.util.Map; import java.util.Optional; +import java.util.Properties; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.Future; import java.util.function.Consumer; import lombok.Cleanup; +import lombok.SneakyThrows; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.Producer; @@ -50,17 +57,38 @@ import org.testng.annotations.Test; public class ProxyServiceStarterTest extends MockedPulsarServiceBaseTest { - - public static final String[] ARGS = new String[]{"-c", "./src/test/resources/proxy.conf"}; - protected ProxyServiceStarter serviceStarter; protected String serviceUrl; + private static File proxyConfFileForTests; + + @SneakyThrows + public static String[] getArgs() { + if (proxyConfFileForTests == null) { + // load the properties from the proxy.conf file + Properties properties = new Properties(); + try (InputStream inputStream = new FileInputStream("../conf/proxy.conf")) { + properties.load(inputStream); + } + // set dummy values for the required properties so that validation is passed + properties.setProperty("brokerServiceURL", "pulsar://0.0.0.0:0"); + properties.setProperty("brokerWebServiceURL", "http://0.0.0.0:0"); + // change keepAliveIntervalSeconds default value so that it's possible to validate that it's configured + properties.setProperty("keepAliveIntervalSeconds", "25"); + // write the properties to a temporary file + proxyConfFileForTests = File.createTempFile("proxy", ".conf"); + proxyConfFileForTests.deleteOnExit(); + try (OutputStream out = new FileOutputStream(proxyConfFileForTests)) { + properties.store(out, null); + } + } + return new String[] { "-c", proxyConfFileForTests.getAbsolutePath() }; + } @Override @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS, null, true); + serviceStarter = new ProxyServiceStarter(getArgs(), null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); serviceStarter.getConfig().setWebServicePort(Optional.of(0)); @@ -99,6 +127,11 @@ public void testProducer() throws Exception { } } + @Test + public void testKeepAliveIntervalSecondsIsConfigured() throws Exception { + assertEquals(serviceStarter.getConfig().getKeepAliveIntervalSeconds(), 25); + } + @Test public void testProduceAndConsumeMessageWithWebsocket() throws Exception { HttpClient producerClient = new HttpClient(); @@ -175,7 +208,7 @@ public void testProxyClientAuthentication() throws Exception { - ProxyServiceStarter serviceStarter = new ProxyServiceStarter(ARGS, null, true); + ProxyServiceStarter serviceStarter = new ProxyServiceStarter(getArgs(), null, true); initConfig.accept(serviceStarter.getConfig()); // ProxyServiceStarter will throw an exception when Authentication#start is failed serviceStarter.getConfig().setBrokerClientAuthenticationPlugin(ExceptionAuthentication1.class.getName()); @@ -187,7 +220,7 @@ public void testProxyClientAuthentication() throws Exception { assertTrue(serviceStarter.getProxyClientAuthentication() instanceof ExceptionAuthentication1); } - serviceStarter = new ProxyServiceStarter(ARGS, null, true); + serviceStarter = new ProxyServiceStarter(getArgs(), null, true); initConfig.accept(serviceStarter.getConfig()); // ProxyServiceStarter will throw an exception when Authentication#start and Authentication#close are failed serviceStarter.getConfig().setBrokerClientAuthenticationPlugin(ExceptionAuthentication2.class.getName()); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java index 8246e974f6f06..96e9593116e1f 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java @@ -18,6 +18,15 @@ */ package org.apache.pulsar.proxy.server; +import static org.apache.pulsar.proxy.server.ProxyServiceStarterTest.getArgs; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import java.net.URI; +import java.nio.ByteBuffer; +import java.util.Base64; +import java.util.Optional; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.Future; import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.Producer; @@ -35,17 +44,6 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.net.URI; -import java.nio.ByteBuffer; -import java.util.Base64; -import java.util.Optional; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.Future; - -import static org.apache.pulsar.proxy.server.ProxyServiceStarterTest.ARGS; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; - public class ProxyServiceTlsStarterTest extends MockedPulsarServiceBaseTest { private ProxyServiceStarter serviceStarter; private String serviceUrl; @@ -55,7 +53,7 @@ public class ProxyServiceTlsStarterTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS, null, true); + serviceStarter = new ProxyServiceStarter(getArgs(), null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); diff --git a/pulsar-proxy/src/test/resources/proxy.conf b/pulsar-proxy/src/test/resources/proxy.conf deleted file mode 100644 index aec9f5ee1c5e9..0000000000000 --- a/pulsar-proxy/src/test/resources/proxy.conf +++ /dev/null @@ -1,250 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -### --- Broker Discovery --- ### - -# The metadata store URL -# Examples: -# * zk:my-zk-1:2181,my-zk-2:2181,my-zk-3:2181 -# * my-zk-1:2181,my-zk-2:2181,my-zk-3:2181 (will default to ZooKeeper when the schema is not specified) -# * zk:my-zk-1:2181,my-zk-2:2181,my-zk-3:2181/my-chroot-path (to add a ZK chroot path) -metadataStoreUrl= - -# The metadata store URL for the configuration data. If empty, we fall back to use metadataStoreUrl -configurationMetadataStoreUrl= - -# if Service Discovery is Disabled this url should point to the discovery service provider. -brokerServiceURL=pulsar://0.0.0.0:0 -brokerServiceURLTLS= - -# These settings are unnecessary if `zookeeperServers` is specified -brokerWebServiceURL=http://0.0.0.0:0 -brokerWebServiceURLTLS= - -# If function workers are setup in a separate cluster, configure the following 2 settings -# to point to the function workers cluster -functionWorkerWebServiceURL= -functionWorkerWebServiceURLTLS= - -# ZooKeeper session timeout (in milliseconds) -zookeeperSessionTimeoutMs=30000 - -# ZooKeeper cache expiry time in seconds -zooKeeperCacheExpirySeconds=300 - -### --- Server --- ### - -# Hostname or IP address the service binds on, default is 0.0.0.0. -bindAddress=0.0.0.0 - -# Hostname or IP address the service advertises to the outside world. -# If not set, the value of `InetAddress.getLocalHost().getCanonicalHostName()` is used. -advertisedAddress= - -# Enable or disable the HAProxy protocol. -haProxyProtocolEnabled=false - -# Enables zero-copy transport of data across network interfaces using the splice system call. -# Zero copy mode cannot be used when TLS is enabled or when proxyLogLevel is > 0. -proxyZeroCopyModeEnabled=true - -# The port to use for server binary Protobuf requests -servicePort=6650 - -# The port to use to server binary Protobuf TLS requests -servicePortTls= - -# Port that discovery service listen on -webServicePort=8080 - -# Port to use to server HTTPS request -webServicePortTls= - -# Path for the file used to determine the rotation status for the proxy instance when responding -# to service discovery health checks -statusFilePath= - -# Proxy log level, default is 0. -# 0: Do not log any tcp channel info -# 1: Parse and log any tcp channel info and command info without message body -# 2: Parse and log channel info, command info and message body -proxyLogLevel=0 - -### ---Authorization --- ### - -# Role names that are treated as "super-users," meaning that they will be able to perform all admin -# operations and publish/consume to/from all topics (as a comma-separated list) -superUserRoles= - -# Whether authorization is enforced by the Pulsar proxy -authorizationEnabled=false - -# Authorization provider as a fully qualified class name -authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider - -# Whether client authorization credentials are forwared to the broker for re-authorization. -# Authentication must be enabled via authenticationEnabled=true for this to take effect. -forwardAuthorizationCredentials=false - -### --- Authentication --- ### - -# Whether authentication is enabled for the Pulsar proxy -authenticationEnabled=false - -# Authentication provider name list (a comma-separated list of class names) -authenticationProviders= - -# When this parameter is not empty, unauthenticated users perform as anonymousUserRole -anonymousUserRole= - -### --- Client Authentication --- ### - -# The three brokerClient* authentication settings below are for the proxy itself and determine how it -# authenticates with Pulsar brokers - -# The authentication plugin used by the Pulsar proxy to authenticate with Pulsar brokers -brokerClientAuthenticationPlugin= - -# The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers -brokerClientAuthenticationParameters= - -# The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers -brokerClientTrustCertsFilePath= - -# Whether TLS is enabled when communicating with Pulsar brokers -tlsEnabledWithBroker=false - -# Tls cert refresh duration in seconds (set 0 to check on every new connection) -tlsCertRefreshCheckDurationSec=300 - -##### --- Rate Limiting --- ##### - -# Max concurrent inbound connections. The proxy will reject requests beyond that. -maxConcurrentInboundConnections=10000 - -# Max concurrent outbound connections. The proxy will error out requests beyond that. -maxConcurrentLookupRequests=50000 - -##### --- TLS --- ##### - -# Deprecated - use servicePortTls and webServicePortTls instead -tlsEnabledInProxy=false - -# Path for the TLS certificate file -tlsCertificateFilePath= - -# Path for the TLS private key file -tlsKeyFilePath= - -# Path for the trusted TLS certificate file. -# This cert is used to verify that any certs presented by connecting clients -# are signed by a certificate authority. If this verification -# fails, then the certs are untrusted and the connections are dropped. -tlsTrustCertsFilePath= - -# Accept untrusted TLS certificate from client. -# If true, a client with a cert which cannot be verified with the -# 'tlsTrustCertsFilePath' cert will allowed to connect to the server, -# though the cert will not be used for client authentication. -tlsAllowInsecureConnection=false - -# Whether the hostname is validated when the proxy creates a TLS connection with brokers -tlsHostnameVerificationEnabled=false - -# Specify the tls protocols the broker will use to negotiate during TLS handshake -# (a comma-separated list of protocol names). -# Examples:- [TLSv1.3, TLSv1.2] -tlsProtocols= - -# Specify the tls cipher the broker will use to negotiate during TLS Handshake -# (a comma-separated list of ciphers). -# Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256] -tlsCiphers= - -# Whether client certificates are required for TLS. Connections are rejected if the client -# certificate isn't trusted. -tlsRequireTrustedClientCertOnConnect=false - -##### --- HTTP --- ##### - -# Http directs to redirect to non-pulsar services. -httpReverseProxyConfigs= - -# Http output buffer size. The amount of data that will be buffered for http requests -# before it is flushed to the channel. A larger buffer size may result in higher http throughput -# though it may take longer for the client to see data. -# If using HTTP streaming via the reverse proxy, this should be set to the minimum value, 1, -# so that clients see the data as soon as possible. -httpOutputBufferSize=32768 - -# Number of threads to use for HTTP requests processing. Default is -# 2 * Runtime.getRuntime().availableProcessors() -httpNumThreads= - -# Enable the enforcement of limits on the incoming HTTP requests -httpRequestsLimitEnabled=false - -# Max HTTP requests per seconds allowed. The excess of requests will be rejected with HTTP code 429 (Too many requests) -httpRequestsMaxPerSecond=100.0 - - -### --- Token Authentication Provider --- ### - -## Symmetric key -# Configure the secret key to be used to validate auth tokens -# The key can be specified like: -# tokenSecretKey=data:;base64,xxxxxxxxx -# tokenSecretKey=file:///my/secret.key ( Note: key file must be DER-encoded ) -tokenSecretKey= - -## Asymmetric public/private key pair -# Configure the public key to be used to validate auth tokens -# The key can be specified like: -# tokenPublicKey=data:;base64,xxxxxxxxx -# tokenPublicKey=file:///my/public.key ( Note: key file must be DER-encoded ) -tokenPublicKey= - -# The token "claim" that will be interpreted as the authentication "role" or "principal" by AuthenticationProviderToken (defaults to "sub" if blank) -tokenAuthClaim= - -# The token audience "claim" name, e.g. "aud", that will be used to get the audience from token. -# If not set, audience will not be verified. -tokenAudienceClaim= - -# The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this. -tokenAudience= - -### --- WebSocket config variables --- ### - -# Enable or disable the WebSocket servlet. -webSocketServiceEnabled=false - -# Name of the cluster to which this broker belongs to -clusterName= - -### --- Deprecated config variables --- ### - -# Deprecated. Use configurationStoreServers -globalZookeeperServers= - -# The ZooKeeper quorum connection string (as a comma-separated list) -zookeeperServers= - -# Configuration store connection string (as a comma-separated list) -configurationStoreServers= From 495d9d6087d0bddf07819aa463e3ade3a73638b9 Mon Sep 17 00:00:00 2001 From: Jiawen Wang <74594733+summeriiii@users.noreply.github.com> Date: Fri, 14 Feb 2025 17:56:45 +0800 Subject: [PATCH 26/44] [fix][broker] Fix incorrect blockedConsumerOnUnackedMsgs value when maxUnackedMessagesPerConsumer is 1 (#23796) (cherry picked from commit 5443c69d84818cb4a49704f7ab7dbccf65b2179a) (cherry picked from commit b098772abd2b331866889359fc892837946df88c) --- .../pulsar/broker/service/Consumer.java | 7 +- .../broker/service/BrokerServiceTest.java | 101 ++++++++++++++++++ 2 files changed, 107 insertions(+), 1 deletion(-) 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 80db47eda1c5d..3338812ccaa54 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 @@ -537,6 +537,7 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map // consumer can start again consuming messages int unAckedMsgs = UNACKED_MESSAGES_UPDATER.get(ackOwnedConsumer); @@ -999,7 +1005,6 @@ private boolean removePendingAcks(Consumer ackOwnedConsumer, PositionImpl positi ackOwnedConsumer.blockedConsumerOnUnackedMsgs = false; flowConsumerBlockedPermits(ackOwnedConsumer); } - return true; } public ConcurrentLongLongPairHashMap getPendingAcks() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 0e1fe3aa535d3..34ddc6d9466c8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -1664,6 +1664,107 @@ public void testDynamicConfigurationsForceDeleteTenantAllowed() throws Exception }); } + @Test + public void testIsSystemTopicAllowAutoTopicCreationAsync() throws Exception { + BrokerService brokerService = pulsar.getBrokerService(); + assertFalse(brokerService.isAllowAutoTopicCreationAsync( + ServiceUnitStateChannelImpl.TOPIC).get()); + assertTrue(brokerService.isAllowAutoTopicCreationAsync( + "persistent://pulsar/system/my-system-topic").get()); + } + + @Test + public void testDuplicateAcknowledgement() throws Exception { + final String ns = "prop/ns-test"; + + admin.namespaces().createNamespace(ns, 2); + final String topicName = "persistent://prop/ns-test/duplicated-acknowledgement-test"; + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topicName) + .create(); + @Cleanup + Consumer consumer1 = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName("sub-1") + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .isAckReceiptEnabled(true) + .subscribe(); + producer.send("1".getBytes(StandardCharsets.UTF_8)); + Message message = consumer1.receive(); + consumer1.acknowledge(message); + consumer1.acknowledge(message); + assertEquals(admin.topics().getStats(topicName).getSubscriptions() + .get("sub-1").getUnackedMessages(), 0); + } + + @Test + public void testBlockedConsumerOnUnackedMsgs() throws Exception { + final String ns = "prop/ns-test"; + admin.namespaces().createNamespace(ns, 2); + admin.namespaces().setMaxUnackedMessagesPerConsumer(ns, 1); + + final String topicName = "persistent://prop/ns-test/testBlockedConsumerOnUnackedMsgs"; + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topicName) + .create(); + @Cleanup + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName("sub-test") + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .isAckReceiptEnabled(true) + .receiverQueueSize(0) + .subscribe(); + + producer.send("1".getBytes(StandardCharsets.UTF_8)); + producer.send("2".getBytes(StandardCharsets.UTF_8)); + + // 1. receive message + Message message = consumer.receive(); + Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); + + SubscriptionStats subscriptionStats = admin.topics().getStats(topicName).getSubscriptions().get("sub-test"); + assertEquals(subscriptionStats.getUnackedMessages(), 1); + assertTrue(subscriptionStats.getConsumers().get(0).isBlockedConsumerOnUnackedMsgs()); + + // 2、ack this message + consumer.acknowledge(message); + Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); + + subscriptionStats = admin.topics().getStats(topicName).getSubscriptions().get("sub-test"); + assertEquals(subscriptionStats.getUnackedMessages(), 0); + assertFalse(subscriptionStats.getConsumers().get(0).isBlockedConsumerOnUnackedMsgs()); + } + + @Test + public void testUnsubscribeNonDurableSub() throws Exception { + final String ns = "prop/ns-test"; + final String topic = ns + "/testUnsubscribeNonDurableSub"; + + admin.namespaces().createNamespace(ns, 2); + admin.topics().createPartitionedTopic(String.format("persistent://%s", topic), 1); + + pulsarClient.newProducer(Schema.STRING).topic(topic).create().close(); + @Cleanup + Consumer consumer = pulsarClient + .newConsumer(Schema.STRING) + .topic(topic) + .subscriptionMode(SubscriptionMode.NonDurable) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("sub1") + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + try { + consumer.unsubscribe(); + } catch (Exception ex) { + fail("Unsubscribe failed"); + } + } + // this test is disabled since it is flaky @Test(enabled = false) public void testMetricsPersistentTopicLoadFails() throws Exception { From 945c1306299e09103d6d8a5b1a27b45aeb887b39 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Fri, 14 Feb 2025 02:52:13 -0800 Subject: [PATCH 27/44] [fix][meta] Fix ephemeral Zookeeper put which creates a persistent znode (#23984) (cherry picked from commit 40d5af4a43ca508ef496367925868310ea683780) (cherry picked from commit 2cf83894fbd9a5591fab76e6bb37b3075f4a4184) --- .../pulsar/metadata/impl/ZKMetadataStore.java | 4 +-- .../metadata/MetadataStoreExtendedTest.java | 34 +++++++++++++++++++ 2 files changed, 36 insertions(+), 2 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 8cfb23faad368..78e9980bc2187 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -438,8 +438,8 @@ private void internalStorePut(OpPut opPut) { future.completeExceptionally(getException(Code.BADVERSION, opPut.getPath())); } else { // The z-node does not exist, let's create it first - put(opPut.getPath(), opPut.getData(), Optional.of(-1L)).thenAccept( - s -> future.complete(s)) + put(opPut.getPath(), opPut.getData(), Optional.of(-1L), opPut.getOptions()) + .thenAccept(s -> future.complete(s)) .exceptionally(ex -> { if (ex.getCause() instanceof BadVersionException) { // The z-node exist now, let's overwrite it diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java index 9a38cdbcd2f85..a4c937611fd3f 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java @@ -66,4 +66,38 @@ public void sequentialKeys(String provider, Supplier urlSupplier) throws assertNotEquals(seq1, seq2); assertTrue(n1 < n2); } + + @Test(dataProvider = "impl") + public void testPersistentOrEphemeralPut(String provider, Supplier urlSupplier) throws Exception { + final String key1 = newKey(); + MetadataStoreExtended store = MetadataStoreExtended.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + store.put(key1, "value-1".getBytes(), Optional.empty(), EnumSet.noneOf(CreateOption.class)).join(); + var value = store.get(key1).join().get(); + assertEquals(value.getValue(), "value-1".getBytes()); + // assertFalse(value.getStat().isEphemeral()); // Todo : fix zkStat.getEphemeralOwner() != 0 from test zk + assertTrue(value.getStat().isFirstVersion()); + var version = value.getStat().getVersion(); + + store.put(key1, "value-2".getBytes(), Optional.empty(), EnumSet.noneOf(CreateOption.class)).join(); + value = store.get(key1).join().get(); + assertEquals(value.getValue(), "value-2".getBytes()); + //assertFalse(value.getStat().isEphemeral()); // Todo : fix zkStat.getEphemeralOwner() != 0 from test zk + assertEquals(value.getStat().getVersion(), version + 1); + + final String key2 = newKey(); + store.put(key2, "value-4".getBytes(), Optional.empty(), EnumSet.of(CreateOption.Ephemeral)).join(); + value = store.get(key2).join().get(); + assertEquals(value.getValue(), "value-4".getBytes()); + assertTrue(value.getStat().isEphemeral()); + assertTrue(value.getStat().isFirstVersion()); + version = value.getStat().getVersion(); + + + store.put(key2, "value-5".getBytes(), Optional.empty(), EnumSet.of(CreateOption.Ephemeral)).join(); + value = store.get(key2).join().get(); + assertEquals(value.getValue(), "value-5".getBytes()); + assertTrue(value.getStat().isEphemeral()); + assertEquals(value.getStat().getVersion(), version + 1); + } + } From aa5643a51d1a0c9ac33180c020a63694a9c6035f Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Sat, 15 Feb 2025 07:30:56 +0800 Subject: [PATCH 28/44] [fix][broker] fix broker may lost rack information (#23331) Co-authored-by: fanjianye (cherry picked from commit a7d9d8f9ae8b5378cbac278f370f5050fa5aaa42) --- .../bookkeeper/PulsarRegistrationClient.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationClient.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationClient.java index be945d988fb88..89dbf2be990b0 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationClient.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationClient.java @@ -181,8 +181,13 @@ private CompletableFuture>> getBookiesThenFreshCache(Str @Override public CompletableFuture watchWritableBookies(RegistrationListener registrationListener) { writableBookiesWatchers.add(registrationListener); + // trigger all listeners in writableBookiesWatchers one by one. It aims to keep a sync way + // to make sure the previous listener has finished when a new listener is register. + // Though it would bring duplicate trigger listener problem, but since watchWritableBookies + // is only executed when bookieClient construct, the duplicate problem is acceptable. return getWritableBookies() - .thenAcceptAsync(registrationListener::onBookiesChanged, executor); + .thenAcceptAsync(bookies -> + writableBookiesWatchers.forEach(w -> w.onBookiesChanged(bookies)), executor); } @Override @@ -193,8 +198,13 @@ public void unwatchWritableBookies(RegistrationListener registrationListener) { @Override public CompletableFuture watchReadOnlyBookies(RegistrationListener registrationListener) { readOnlyBookiesWatchers.add(registrationListener); + // trigger all listeners in readOnlyBookiesWatchers one by one. It aims to keep a sync way + // to make sure the previous listener has finished when a new listener is register. + // Though it would bring duplicate trigger listener problem, but since watchReadOnlyBookies + // is only executed when bookieClient construct, the duplicate problem is acceptable. return getReadOnlyBookies() - .thenAcceptAsync(registrationListener::onBookiesChanged, executor); + .thenAcceptAsync(bookies -> + readOnlyBookiesWatchers.forEach(w -> w.onBookiesChanged(bookies)), executor); } @Override From d9553aeb3e91b23c77f1ec872438ef78087d448f Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 3 Jun 2024 19:31:15 -0700 Subject: [PATCH 29/44] [improve] Validate user paths in Functions utils (#22833) (cherry picked from commit ca8b465897fd6176b614e2b3f2a841b349037aad) (cherry picked from commit b4b69c8ba8695347000c9fcca2b878159efc359d) --- .../pulsar/broker/web/ExceptionHandler.java | 2 + .../functions/utils/FunctionConfigUtils.java | 14 ++++++- .../filesystem/FileSystemPackagesStorage.java | 42 +++++++++++++------ 3 files changed, 43 insertions(+), 15 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java index b11ec3a8a98db..205e02ed75a2e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java @@ -24,6 +24,7 @@ import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletResponse; import javax.ws.rs.core.Response; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.intercept.InterceptException; import org.apache.pulsar.common.policies.data.ErrorData; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -36,6 +37,7 @@ /** * Exception handler for handle exception. */ +@Slf4j public class ExceptionHandler { public void handle(ServletResponse response, Exception ex) throws IOException { diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java index 3c96837e4374e..45fb4c1cb1ee7 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java @@ -860,14 +860,24 @@ public static void doCommonChecks(FunctionConfig functionConfig) { if (!isEmpty(functionConfig.getPy()) && !org.apache.pulsar.common.functions.Utils .isFunctionPackageUrlSupported(functionConfig.getPy()) && functionConfig.getPy().startsWith(BUILTIN)) { - if (!new File(functionConfig.getPy()).exists()) { + String filename = functionConfig.getPy(); + if (filename.contains("..")) { + throw new IllegalArgumentException("Invalid filename: " + filename); + } + + if (!new File(filename).exists()) { throw new IllegalArgumentException("The supplied python file does not exist"); } } if (!isEmpty(functionConfig.getGo()) && !org.apache.pulsar.common.functions.Utils .isFunctionPackageUrlSupported(functionConfig.getGo()) && functionConfig.getGo().startsWith(BUILTIN)) { - if (!new File(functionConfig.getGo()).exists()) { + String filename = functionConfig.getGo(); + if (filename.contains("..")) { + throw new IllegalArgumentException("Invalid filename: " + filename); + } + + if (!new File(filename).exists()) { throw new IllegalArgumentException("The supplied go file does not exist"); } } diff --git a/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java b/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java index 47d825ea928f4..2bb43bb207203 100644 --- a/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java +++ b/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java @@ -58,7 +58,11 @@ public class FileSystemPackagesStorage implements PackagesStorage { } } - private File getPath(String path) { + private File getPath(String path) throws IOException { + if (path.contains("..")) { + throw new IOException("Invalid path: " + path); + } + File f = Paths.get(storagePath.toString(), path).toFile(); if (!f.getParentFile().exists()) { if (!f.getParentFile().mkdirs()) { @@ -119,28 +123,40 @@ public CompletableFuture readAsync(String path, OutputStream outputStream) @Override public CompletableFuture deleteAsync(String path) { - if (getPath(path).delete()) { - return CompletableFuture.completedFuture(null); - } else { - CompletableFuture f = new CompletableFuture<>(); - f.completeExceptionally(new IOException("Failed to delete file at " + path)); - return f; + try { + if (getPath(path).delete()) { + return CompletableFuture.completedFuture(null); + } else { + CompletableFuture f = new CompletableFuture<>(); + f.completeExceptionally(new IOException("Failed to delete file at " + path)); + return f; + } + } catch (IOException e) { + return CompletableFuture.failedFuture(e); } } @Override public CompletableFuture> listAsync(String path) { - String[] files = getPath(path).list(); - if (files == null) { - return CompletableFuture.completedFuture(Collections.emptyList()); - } else { - return CompletableFuture.completedFuture(Arrays.asList(files)); + try { + String[] files = getPath(path).list(); + if (files == null) { + return CompletableFuture.completedFuture(Collections.emptyList()); + } else { + return CompletableFuture.completedFuture(Arrays.asList(files)); + } + } catch (IOException e) { + return CompletableFuture.failedFuture(e); } } @Override public CompletableFuture existAsync(String path) { - return CompletableFuture.completedFuture(getPath(path).exists()); + try { + return CompletableFuture.completedFuture(getPath(path).exists()); + } catch (IOException e) { + return CompletableFuture.failedFuture(e); + } } @Override From d1b73c1a41c629cc72c23ad06b7786bbb019ee19 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 14 Feb 2025 19:18:06 +0800 Subject: [PATCH 30/44] [fix][test] fix flaky testNegativeAcksWithBackoff when batch enabled. (#23986) (cherry picked from commit eb7a4f36da3711b9b93527d0bfe26acad1d64b1d) (cherry picked from commit e729c90d8b0a50b03945371ed509fdfa8dc89081) --- .../org/apache/pulsar/client/impl/NegativeAcksTest.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java index 393fc924a2ae0..17f91bc68fa94 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java @@ -255,9 +255,16 @@ public void testNegativeAcksWithBackoff(boolean batching, boolean usePartitions, long firstReceivedAt = System.currentTimeMillis(); long expectedTotalRedeliveryDelay = 0; for (int i = 0; i < redeliverCount; i++) { + Message msg = null; for (int j = 0; j < N; j++) { - Message msg = consumer.receive(); + msg = consumer.receive(); log.info("Received message {}", msg.getValue()); + if (!batching) { + consumer.negativeAcknowledge(msg); + } + } + if (batching) { + // for batching, we only need to nack one message in the batch to trigger redelivery consumer.negativeAcknowledge(msg); } expectedTotalRedeliveryDelay += backoff.next(i); From 5a23178070138cb82f5d8e5f1d66570ec5df8dc3 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 14 Feb 2025 08:54:59 -0800 Subject: [PATCH 31/44] [fix] fix for code scanning alert no. 48: Uncontrolled data used in path expression (#23985) Co-authored-by: Copilot Autofix powered by AI <62310815+github-advanced-security[bot]@users.noreply.github.com> (cherry picked from commit 58120841ec74373b5ea718dd0325d4542dde4330) (cherry picked from commit 8ee80ffec634293a635d55f1e777849ab22f000d) --- .../storage/filesystem/FileSystemPackagesStorage.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java b/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java index 2bb43bb207203..8bf7851fc8d63 100644 --- a/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java +++ b/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java @@ -59,11 +59,14 @@ public class FileSystemPackagesStorage implements PackagesStorage { } private File getPath(String path) throws IOException { - if (path.contains("..")) { + // Normalize the path to remove any redundant path elements + File f = Paths.get(storagePath.toString(), path).normalize().toFile(); + + // Ensure the normalized path is still within the storagePath + if (!f.getAbsolutePath().startsWith(storagePath.getAbsolutePath())) { throw new IOException("Invalid path: " + path); } - File f = Paths.get(storagePath.toString(), path).toFile(); if (!f.getParentFile().exists()) { if (!f.getParentFile().mkdirs()) { throw new RuntimeException("Failed to create parent dirs for " + path); From c5468252789153dcca2e8d371381dac6e7f53e19 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 17 Feb 2025 14:46:45 +0800 Subject: [PATCH 32/44] [fix][build] Add develops for buildtools (#23992) Signed-off-by: Zixuan Liu (cherry picked from commit 40b96de0410c19551d1700501e28d8a8c4a63336) (cherry picked from commit 9cdf9ade1700567d5f811ce84f16a86dc9b749f2) --- buildtools/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 47950cfd0c772..980f0ec048efe 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -35,6 +35,13 @@ jar Pulsar Build Tools + + + Apache Pulsar developers + http://pulsar.apache.org/ + + + 2025-01-20T05:51:16Z 1.8 From 1320353139eeff3c85795eafb799ead97d8fa105 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Sat, 14 Dec 2024 14:49:10 +0800 Subject: [PATCH 33/44] [fix][broker] Fix the retry mechanism in `MetadataCache#readModifyUpdateOrCreate` (#23686) The method `MetadataCache#readModifyUpdateOrCreate` should handle the BadVersionException by retrying the modification process, as already noted in the Java documentation: "The modify function can potentially be called multiple times due to concurrent updates." Currently, `MetadataCache#readModifyUpdateOrCreate` does not catch the BadVersionException on the second attempt, allowing the exception to be passed to the caller. This issue can be easily reproduced by increasing concurrent futures in the test `MetadataCacheTest#readModifyUpdateBadVersionRetry`. The current retry implementation is incorrect and lacks a backoff mechanism, which could lead to too many requests to the metadata store. - Correct the retry process in `MetadataCache#readModifyUpdateOrCreate` to ensure BadVersionException is caught during each retry. - Implement a retry backoff mechanism in `MetadataCache#readModifyUpdateOrCreate` to manage the frequency of retries effectively. - Add new config `retryBackoff` to the MetadataCacheConfig to control the MetadataCache retry backoff. - Respective the `metadataStoreOperationTimeoutSeconds` for the MetadataCache retry (cherry picked from commit 0ae3f9def9c4f35d9cd5bd590a0110bc76f13cab) (cherry picked from commit b56dfc7fa56b8ad15cfe524101b587958aa16b7c) --- .../broker/resources/BaseResources.java | 11 +- .../apache/pulsar/common/util/Backoff.java | 6 -- .../pulsar/common/util/BackoffTest.java | 1 + .../metadata/api/MetadataCacheConfig.java | 8 ++ .../cache/impl/MetadataCacheImpl.java | 42 ++++++-- .../metadata/impl/AbstractMetadataStore.java | 6 +- .../pulsar/metadata/MetadataCacheTest.java | 100 +++++++++++++++--- 7 files changed, 136 insertions(+), 38 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java index 00e381e07292f..f31e5a6b78a65 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java @@ -34,6 +34,7 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.metadata.api.MetadataCache; +import org.apache.pulsar.metadata.api.MetadataCacheConfig; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -58,13 +59,19 @@ public class BaseResources { public BaseResources(MetadataStore store, Class clazz, int operationTimeoutSec) { this.store = store; - this.cache = store.getMetadataCache(clazz); + this.cache = store.getMetadataCache(clazz, MetadataCacheConfig.builder() + .retryBackoff(MetadataCacheConfig.DEFAULT_RETRY_BACKOFF_BUILDER.setMandatoryStop(operationTimeoutSec, + TimeUnit.SECONDS)) + .build()); this.operationTimeoutSec = operationTimeoutSec; } public BaseResources(MetadataStore store, TypeReference typeRef, int operationTimeoutSec) { this.store = store; - this.cache = store.getMetadataCache(typeRef); + this.cache = store.getMetadataCache(typeRef, MetadataCacheConfig.builder() + .retryBackoff(MetadataCacheConfig.DEFAULT_RETRY_BACKOFF_BUILDER.setMandatoryStop(operationTimeoutSec, + TimeUnit.SECONDS)) + .build()); this.operationTimeoutSec = operationTimeoutSec; } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/Backoff.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/Backoff.java index 4eab85f3c41be..d7381792f60e8 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/Backoff.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/Backoff.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.common.util; -import com.google.common.annotations.VisibleForTesting; import java.time.Clock; import java.util.Random; import java.util.concurrent.TimeUnit; @@ -95,11 +94,6 @@ public void reset() { this.mandatoryStopMade = false; } - @VisibleForTesting - long getFirstBackoffTimeInMillis() { - return firstBackoffTimeInMillis; - } - public static boolean shouldBackoff(long initialTimestamp, TimeUnit unitInitial, int failedAttempts, long defaultInterval, long maxBackoffInterval) { long initialTimestampInNano = unitInitial.toNanos(initialTimestamp); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/BackoffTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/BackoffTest.java index b3786236a70ef..ccac4d0e5a13b 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/BackoffTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/BackoffTest.java @@ -136,6 +136,7 @@ public void mandatoryStopTest() { // would have been 1600 w/o the mandatory stop assertTrue(withinTenPercentAndDecrementTimer(backoff, 400)); + assertTrue(backoff.isMandatoryStopMade()); Mockito.when(mockClock.millis()).thenReturn(1900L); assertTrue(withinTenPercentAndDecrementTimer(backoff, 3200)); Mockito.when(mockClock.millis()).thenReturn(3200L); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java index 55b159071fda4..db49229e814c7 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java @@ -22,6 +22,7 @@ import lombok.Builder; import lombok.Getter; import lombok.ToString; +import org.apache.pulsar.common.util.BackoffBuilder; /** * The configuration builder for a {@link MetadataCache} config. @@ -31,6 +32,10 @@ @ToString public class MetadataCacheConfig { private static final long DEFAULT_CACHE_REFRESH_TIME_MILLIS = TimeUnit.MINUTES.toMillis(5); + public static final BackoffBuilder DEFAULT_RETRY_BACKOFF_BUILDER = + new BackoffBuilder().setInitialTime(5, TimeUnit.MILLISECONDS) + .setMax(3, TimeUnit.SECONDS) + .setMandatoryStop(30, TimeUnit.SECONDS); /** * Specifies that active entries are eligible for automatic refresh once a fixed duration has @@ -47,4 +52,7 @@ public class MetadataCacheConfig { */ @Builder.Default private final long expireAfterWriteMillis = 2 * DEFAULT_CACHE_REFRESH_TIME_MILLIS; + + @Builder.Default + private final BackoffBuilder retryBackoff = DEFAULT_RETRY_BACKOFF_BUILDER; } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java index b9051a7dc7df4..1f7034d41857b 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java @@ -29,13 +29,16 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.metadata.api.CacheGetResult; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataCache; @@ -54,20 +57,27 @@ public class MetadataCacheImpl implements MetadataCache, Consumer serde; + private final ScheduledExecutorService executor; + private final MetadataCacheConfig cacheConfig; private final AsyncLoadingCache>> objCache; - public MetadataCacheImpl(MetadataStore store, TypeReference typeRef, MetadataCacheConfig cacheConfig) { - this(store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig); + public MetadataCacheImpl(MetadataStore store, TypeReference typeRef, MetadataCacheConfig cacheConfig, + ScheduledExecutorService executor) { + this(store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig, executor); } - public MetadataCacheImpl(MetadataStore store, JavaType type, MetadataCacheConfig cacheConfig) { - this(store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig); + public MetadataCacheImpl(MetadataStore store, JavaType type, MetadataCacheConfig cacheConfig, + ScheduledExecutorService executor) { + this(store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig, executor); } - public MetadataCacheImpl(MetadataStore store, MetadataSerde serde, MetadataCacheConfig cacheConfig) { + public MetadataCacheImpl(MetadataStore store, MetadataSerde serde, MetadataCacheConfig cacheConfig, + ScheduledExecutorService executor) { this.store = store; this.serde = serde; + this.cacheConfig = cacheConfig; + this.executor = executor; Caffeine cacheBuilder = Caffeine.newBuilder(); if (cacheConfig.getRefreshAfterWriteMillis() > 0) { @@ -292,22 +302,34 @@ public void accept(Notification t) { } } - private CompletableFuture executeWithRetry(Supplier> op, String key) { - CompletableFuture result = new CompletableFuture<>(); + private void execute(Supplier> op, String key, CompletableFuture result, Backoff backoff) { op.get().thenAccept(result::complete).exceptionally((ex) -> { if (ex.getCause() instanceof BadVersionException) { // if resource is updated by other than metadata-cache then metadata-cache will get bad-version // exception. so, try to invalidate the cache and try one more time. objCache.synchronous().invalidate(key); - op.get().thenAccept(result::complete).exceptionally((ex1) -> { - result.completeExceptionally(ex1.getCause()); + long elapsed = System.currentTimeMillis() - backoff.getFirstBackoffTimeInMillis(); + if (backoff.isMandatoryStopMade()) { + result.completeExceptionally(new TimeoutException( + String.format("Timeout to update key %s. Elapsed time: %d ms", key, elapsed))); return null; - }); + } + final var next = backoff.next(); + log.info("Update key {} conflicts. Retrying in {} ms. Mandatory stop: {}. Elapsed time: {} ms", key, + next, backoff.isMandatoryStopMade(), elapsed); + executor.schedule(() -> execute(op, key, result, backoff), next, + TimeUnit.MILLISECONDS); return null; } result.completeExceptionally(ex.getCause()); return null; }); + } + + private CompletableFuture executeWithRetry(Supplier> op, String key) { + final var backoff = cacheConfig.getRetryBackoff().create(); + CompletableFuture result = new CompletableFuture<>(); + execute(op, key, result, backoff); return result; } } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java index dcd5bfde008f7..1c2ef4dad3cee 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java @@ -235,21 +235,21 @@ protected boolean shouldIgnoreEvent(MetadataEvent event, GetResult existingValue @Override public MetadataCache getMetadataCache(Class clazz, MetadataCacheConfig cacheConfig) { MetadataCacheImpl metadataCache = new MetadataCacheImpl(this, - TypeFactory.defaultInstance().constructSimpleType(clazz, null), cacheConfig); + TypeFactory.defaultInstance().constructSimpleType(clazz, null), cacheConfig, this.executor); metadataCaches.add(metadataCache); return metadataCache; } @Override public MetadataCache getMetadataCache(TypeReference typeRef, MetadataCacheConfig cacheConfig) { - MetadataCacheImpl metadataCache = new MetadataCacheImpl(this, typeRef, cacheConfig); + MetadataCacheImpl metadataCache = new MetadataCacheImpl(this, typeRef, cacheConfig, this.executor); metadataCaches.add(metadataCache); return metadataCache; } @Override public MetadataCache getMetadataCache(MetadataSerde serde, MetadataCacheConfig cacheConfig) { - MetadataCacheImpl metadataCache = new MetadataCacheImpl<>(this, serde, cacheConfig); + MetadataCacheImpl metadataCache = new MetadataCacheImpl<>(this, serde, cacheConfig, this.executor); metadataCaches.add(metadataCache); return metadataCache; } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java index 0c30b238049c0..591c0a23a9bad 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.metadata; +import static org.mockito.ArgumentMatchers.any; +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.assertNotSame; @@ -26,7 +29,9 @@ import static org.testng.Assert.fail; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.core.type.TypeReference; +import com.github.benmanes.caffeine.cache.AsyncLoadingCache; import java.io.IOException; +import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -35,6 +40,8 @@ import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; import lombok.AllArgsConstructor; @@ -43,12 +50,16 @@ import lombok.NoArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.util.BackoffBuilder; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.metadata.api.CacheGetResult; import org.apache.pulsar.metadata.api.MetadataCache; +import org.apache.pulsar.metadata.api.MetadataCacheConfig; import org.apache.pulsar.metadata.api.MetadataSerde; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.MetadataStoreException.AlreadyExistsException; import org.apache.pulsar.metadata.api.MetadataStoreException.ContentDeserializationException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; @@ -57,6 +68,7 @@ import org.apache.pulsar.metadata.api.Stat; import org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl; import org.awaitility.Awaitility; +import org.mockito.stubbing.Answer; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -485,31 +497,74 @@ public void readModifyUpdate(String provider, Supplier urlSupplier) thro public void readModifyUpdateBadVersionRetry() throws Exception { String url = zks.getConnectionString(); @Cleanup - MetadataStore sourceStore1 = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); - MetadataStore sourceStore2 = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); + MetadataStore store = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); - MetadataCache objCache1 = sourceStore1.getMetadataCache(MyClass.class); - MetadataCache objCache2 = sourceStore2.getMetadataCache(MyClass.class); + MetadataCache cache = store.getMetadataCache(MyClass.class); String key1 = newKey(); MyClass value1 = new MyClass("a", 1); - objCache1.create(key1, value1).join(); - assertEquals(objCache1.get(key1).join().get().b, 1); - - CompletableFuture future1 = objCache1.readModifyUpdate(key1, v -> { - return new MyClass(v.a, v.b + 1); - }); + cache.create(key1, value1).join(); + assertEquals(cache.get(key1).join().get().b, 1); - CompletableFuture future2 = objCache2.readModifyUpdate(key1, v -> { - return new MyClass(v.a, v.b + 1); - }); + final var futures = new ArrayList>(); + final var sourceStores = new ArrayList(); - MyClass myClass1 = future1.join(); - assertEquals(myClass1.b, 2); + for (int i = 0; i < 20; i++) { + final var sourceStore = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); + sourceStores.add(sourceStore); + final var objCache = sourceStore.getMetadataCache(MyClass.class); + futures.add(objCache.readModifyUpdate(key1, v -> new MyClass(v.a, v.b + 1))); + } + FutureUtil.waitForAll(futures).join(); + for (var sourceStore : sourceStores) { + sourceStore.close(); + } + } - MyClass myClass2 = future2.join(); - assertEquals(myClass2.b, 3); + @Test + public void readModifyUpdateOrCreateRetryTimeout() throws Exception { + String url = zks.getConnectionString(); + @Cleanup + MetadataStore store = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); + + MetadataCache cache = store.getMetadataCache(MyClass.class, MetadataCacheConfig.builder() + .retryBackoff(new BackoffBuilder() + .setInitialTime(5, TimeUnit.MILLISECONDS) + .setMax(1, TimeUnit.SECONDS) + .setMandatoryStop(3, TimeUnit.SECONDS)).build()); + + Field metadataCacheField = cache.getClass().getDeclaredField("objCache"); + metadataCacheField.setAccessible(true); + var objCache = metadataCacheField.get(cache); + var spyObjCache = (AsyncLoadingCache) spy(objCache); + doAnswer((Answer>) invocation -> CompletableFuture.failedFuture( + new MetadataStoreException.BadVersionException(""))).when(spyObjCache).get(any()); + metadataCacheField.set(cache, spyObjCache); + + // Test three times to ensure that the retry works each time. + for (int i = 0; i < 3; i++) { + var start = System.currentTimeMillis(); + boolean timeouted = false; + try { + cache.readModifyUpdateOrCreate(newKey(), Optional::get).join(); + } catch (CompletionException e) { + if (e.getCause() instanceof TimeoutException) { + var elapsed = System.currentTimeMillis() - start; + // Since we reduce the wait time by a random amount for each retry, the total elapsed time should be + // mandatoryStopTime - maxTime * 0.9, which is 2900ms. + assertTrue(elapsed >= 2900L, + "The elapsed time should be greater than the timeout. But now it's " + elapsed); + // The elapsed time should be less than the timeout. The 1.5 factor allows for some extra time. + assertTrue(elapsed < 3000L * 1.5, + "The retry should have been stopped after the timeout. But now it's " + elapsed); + timeouted = true; + } else { + fail("Should have failed with TimeoutException, but failed with " + e.getCause()); + } + } + assertTrue(timeouted, "Should have failed with TimeoutException, but succeeded"); + } } @Test(dataProvider = "impl") @@ -596,4 +651,15 @@ public CustomClass deserialize(String path, byte[] content, Stat stat) throws IO assertEquals(res.getValue().b, 2); assertEquals(res.getValue().path, key1); } + + @Test + public void testDefaultMetadataCacheConfig() { + final var config = MetadataCacheConfig.builder().build(); + assertEquals(config.getRefreshAfterWriteMillis(), TimeUnit.MINUTES.toMillis(5)); + assertEquals(config.getExpireAfterWriteMillis(), TimeUnit.MINUTES.toMillis(10)); + final var backoff = config.getRetryBackoff().create(); + assertEquals(backoff.getInitial(), 5); + assertEquals(backoff.getMax(), 3000); + assertEquals(backoff.getMandatoryStop(), 30_000); + } } From c4ad461da9f8a13bf147e179b1dea8d3f7264f44 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 18:54:43 +0200 Subject: [PATCH 34/44] [fix][meta] Fix ephemeral handling of ZK nodes and fix MockZooKeeper ephemeral and ZK stat handling (#23988) (cherry picked from commit df5197212e8806c7d1907dedfcdfd9e40a4f0ea5) (cherry picked from commit 67ae209e995017fe41ac6a9c998aa7e63576c837) --- .../loadbalance/impl/LoadManagerShared.java | 4 +- .../impl/ModularLoadManagerImpl.java | 2 +- .../pulsar/broker/web/PulsarWebResource.java | 2 +- .../common/naming/NamespaceBundles.java | 8 +- .../auth/MockedPulsarServiceBaseTest.java | 27 +- .../broker/testcontext/PulsarTestContext.java | 93 +- .../client/api/BrokerServiceLookupTest.java | 113 +- .../client/api/ProducerConsumerBase.java | 2 +- .../pulsar/metadata/impl/ZKMetadataStore.java | 14 +- .../BookKeeperClusterTestCase.java | 4 + .../metadata/BaseMetadataStoreTest.java | 136 +- .../apache/pulsar/metadata/CounterTest.java | 3 +- .../pulsar/metadata/MetadataCacheTest.java | 29 +- .../metadata/MetadataStoreExtendedTest.java | 6 +- .../pulsar/metadata/MetadataStoreTest.java | 28 +- .../MockZooKeeperMetadataStoreProvider.java | 49 + .../bookkeeper/LedgerManagerIteratorTest.java | 18 +- .../LedgerUnderreplicationManagerTest.java | 8 +- .../PulsarLedgerIdGeneratorTest.java | 2 +- .../impl/MetadataStoreFactoryImplTest.java | 21 +- .../org/apache/zookeeper/MockZooKeeper.java | 1199 +++++++++-------- .../zookeeper/MockZooKeeperSession.java | 201 ++- 22 files changed, 1210 insertions(+), 759 deletions(-) create mode 100644 pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MockZooKeeperMetadataStoreProvider.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java index 3d627db6cfa9e..db2122331a50b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java @@ -305,7 +305,7 @@ public static void fillNamespaceToBundlesMap(final Set bundles, public static String getBundleRangeFromBundleName(String bundleName) { // the bundle format is property/cluster/namespace/0x00000000_0xFFFFFFFF int pos = bundleName.lastIndexOf("/"); - checkArgument(pos != -1); + checkArgument(pos != -1, "Invalid bundle name format: %s", bundleName); return bundleName.substring(pos + 1); } @@ -313,7 +313,7 @@ public static String getBundleRangeFromBundleName(String bundleName) { public static String getNamespaceNameFromBundleName(String bundleName) { // the bundle format is property/cluster/namespace/0x00000000_0xFFFFFFFF int pos = bundleName.lastIndexOf('/'); - checkArgument(pos != -1); + checkArgument(pos != -1, "Invalid bundle name format: %s", bundleName); return bundleName.substring(0, pos); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 112ada3fd942b..13866ec4ccf27 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -612,7 +612,7 @@ private void updateBundleData() { for (String bundle : bundleData.keySet()) { if (!activeBundles.contains(bundle)){ bundleData.remove(bundle); - if (pulsar.getLeaderElectionService().isLeader()){ + if (pulsar.getLeaderElectionService() != null && pulsar.getLeaderElectionService().isLeader()){ deleteBundleDataFromMetadataStore(bundle); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java index dafad019613a6..fde96d600401a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java @@ -1013,7 +1013,7 @@ protected static boolean isLeaderBroker(PulsarService pulsar) { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return true; } - return pulsar.getLeaderElectionService().isLeader(); + return pulsar.getLeaderElectionService() != null && pulsar.getLeaderElectionService().isLeader(); } public void validateTenantOperation(String tenant, TenantOperation operation) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java index fa7baeaa6067b..972eead3389bc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java @@ -108,9 +108,11 @@ public int size() { public void validateBundle(NamespaceBundle nsBundle) throws Exception { int idx = Arrays.binarySearch(partitions, nsBundle.getLowerEndpoint()); - checkArgument(idx >= 0, "Cannot find bundle in the bundles list"); - checkArgument(nsBundle.getUpperEndpoint().equals(bundles.get(idx).getUpperEndpoint()), - "Invalid upper boundary for bundle"); + checkArgument(idx >= 0, "Cannot find bundle %s in the bundles list", nsBundle); + NamespaceBundle foundBundle = bundles.get(idx); + Long upperEndpoint = foundBundle.getUpperEndpoint(); + checkArgument(nsBundle.getUpperEndpoint().equals(upperEndpoint), + "Invalid upper boundary for bundle %s. Expected upper boundary of %s", nsBundle, foundBundle); } public NamespaceBundle getFullBundle() { 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 ca0f414dd067d..02fa2fe858890 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 @@ -144,6 +144,9 @@ public static String getTlsFileForClient(String name) { protected boolean enableBrokerInterceptor = false; + // Set to true in test's constructor to use a real Zookeeper (TestZKServer) + protected boolean useTestZookeeper; + public MockedPulsarServiceBaseTest() { resetConfig(); } @@ -328,7 +331,14 @@ protected void afterPulsarStart(PulsarService pulsar) throws Exception { * @throws Exception if an error occurs */ protected void restartBroker() throws Exception { + restartBroker(null); + } + + protected void restartBroker(Consumer configurationChanger) throws Exception { stopBroker(); + if (configurationChanger != null) { + configurationChanger.accept(conf); + } startBroker(); } @@ -419,7 +429,6 @@ protected PulsarTestContext.Builder createPulsarTestContextBuilder(ServiceConfig PulsarTestContext.Builder builder = PulsarTestContext.builder() .spyByDefault() .config(conf) - .withMockZookeeper(true) .pulsarServiceCustomizer(pulsarService -> { try { beforePulsarStart(pulsarService); @@ -428,9 +437,25 @@ protected PulsarTestContext.Builder createPulsarTestContextBuilder(ServiceConfig } }) .brokerServiceCustomizer(this::customizeNewBrokerService); + configureMetadataStores(builder); return builder; } + /** + * Configures the metadata stores for the PulsarTestContext.Builder instance. + * Set useTestZookeeper to true in the test's constructor to use TestZKServer which is a real ZooKeeper + * implementation. + * + * @param builder the PulsarTestContext.Builder instance to configure + */ + protected void configureMetadataStores(PulsarTestContext.Builder builder) { + if (useTestZookeeper) { + builder.withTestZookeeper(); + } else { + builder.withMockZookeeper(true); + } + } + /** * This method can be used in test classes for creating additional PulsarTestContext instances * that share the same mock ZooKeeper and BookKeeper instances as the main PulsarTestContext instance. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java index 326cfaf2b713d..89f8422138faf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java @@ -19,12 +19,10 @@ package org.apache.pulsar.broker.testcontext; -import com.google.common.util.concurrent.MoreExecutors; import io.netty.channel.EventLoopGroup; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.time.Duration; -import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.function.Consumer; @@ -33,6 +31,7 @@ import lombok.Builder; import lombok.Getter; import lombok.Singular; +import lombok.SneakyThrows; import lombok.ToString; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; @@ -58,6 +57,7 @@ import org.apache.pulsar.compaction.CompactionServiceFactory; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; +import org.apache.pulsar.metadata.TestZKServer; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -65,9 +65,11 @@ import org.apache.pulsar.metadata.impl.MetadataStoreFactoryImpl; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.MockZooKeeper; import org.apache.zookeeper.MockZooKeeperSession; -import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; import org.jetbrains.annotations.NotNull; import org.mockito.Mockito; import org.mockito.internal.util.MockUtil; @@ -154,6 +156,10 @@ public class PulsarTestContext implements AutoCloseable { private final MockZooKeeper mockZooKeeperGlobal; + private final TestZKServer testZKServer; + + private final TestZKServer testZKServerGlobal; + private final SpyConfig spyConfig; private final boolean startable; @@ -381,6 +387,11 @@ public Builder reuseMockBookkeeperAndMetadataStores(PulsarTestContext otherConte if (otherContext.getMockZooKeeperGlobal() != null) { mockZooKeeperGlobal(otherContext.getMockZooKeeperGlobal()); } + } else if (otherContext.getTestZKServer() != null) { + testZKServer(otherContext.getTestZKServer()); + if (otherContext.getTestZKServerGlobal() != null) { + testZKServerGlobal(otherContext.getTestZKServerGlobal()); + } } else { localMetadataStore(NonClosingProxyHandler.createNonClosingProxy(otherContext.getLocalMetadataStore(), MetadataStoreExtended.class @@ -440,17 +451,56 @@ public Builder withMockZookeeper(boolean useSeparateGlobalZk) { } private MockZooKeeper createMockZooKeeper() throws Exception { - MockZooKeeper zk = MockZooKeeper.newInstance(MoreExecutors.newDirectExecutorService()); - List dummyAclList = new ArrayList<>(0); + MockZooKeeper zk = MockZooKeeper.newInstance(); + initializeZookeeper(zk); + registerCloseable(zk::shutdown); + return zk; + } + private static void initializeZookeeper(ZooKeeper zk) throws KeeperException, InterruptedException { ZkUtils.createFullPathOptimistic(zk, "/ledgers/available/192.168.1.1:" + 5000, - "".getBytes(StandardCharsets.UTF_8), dummyAclList, CreateMode.PERSISTENT); + "".getBytes(StandardCharsets.UTF_8), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk.create("/ledgers/LAYOUT", "1\nflat:1".getBytes(StandardCharsets.UTF_8), dummyAclList, + zk.create("/ledgers/LAYOUT", "1\nflat:1".getBytes(StandardCharsets.UTF_8), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } - registerCloseable(zk::shutdown); - return zk; + /** + * Configure this PulsarTestContext to use a test ZooKeeper instance which is + * shared for both the local and configuration metadata stores. + * + * @return the builder + */ + public Builder withTestZookeeper() { + return withTestZookeeper(false); + } + + /** + * Configure this PulsarTestContext to use a test ZooKeeper instance. + * + * @param useSeparateGlobalZk if true, the global (configuration) zookeeper will be a separate instance + * @return the builder + */ + public Builder withTestZookeeper(boolean useSeparateGlobalZk) { + try { + testZKServer(createTestZookeeper()); + if (useSeparateGlobalZk) { + testZKServerGlobal(createTestZookeeper()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + return this; + } + + private TestZKServer createTestZookeeper() throws Exception { + TestZKServer testZKServer = new TestZKServer(); + try (ZooKeeper zkc = new ZooKeeper(testZKServer.getConnectionString(), 5000, event -> { + })) { + initializeZookeeper(zkc); + } + registerCloseable(testZKServer); + return testZKServer; } /** @@ -632,6 +682,20 @@ private void initializeCommonPulsarServices(SpyConfig spyConfig) { configurationMetadataStore(mockZookeeperMetadataStore); } } + } else if (super.testZKServer != null) { + MetadataStoreExtended testZookeeperMetadataStore = + createTestZookeeperMetadataStore(super.testZKServer, MetadataStoreConfig.METADATA_STORE); + if (super.localMetadataStore == null) { + localMetadataStore(testZookeeperMetadataStore); + } + if (super.configurationMetadataStore == null) { + if (super.testZKServerGlobal != null) { + configurationMetadataStore(createTestZookeeperMetadataStore(super.testZKServerGlobal, + MetadataStoreConfig.CONFIGURATION_METADATA_STORE)); + } else { + configurationMetadataStore(testZookeeperMetadataStore); + } + } } else { try { MetadataStoreExtended store = MetadataStoreFactoryImpl.createExtended("memory:local", @@ -676,6 +740,17 @@ private MetadataStoreExtended createMockZookeeperMetadataStore(MockZooKeeper moc return nonClosingProxy; } + @SneakyThrows + private MetadataStoreExtended createTestZookeeperMetadataStore(TestZKServer zkServer, + String metadataStoreName) { + MetadataStoreExtended store = MetadataStoreExtended.create("zk:" + zkServer.getConnectionString(), + MetadataStoreConfig.builder().metadataStoreName(metadataStoreName).build()); + registerCloseable(store); + MetadataStoreExtended nonClosingProxy = + NonClosingProxyHandler.createNonClosingProxy(store, MetadataStoreExtended.class); + return nonClosingProxy; + } + protected abstract void initializePulsarServices(SpyConfig spyConfig, Builder builder); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index f132aef96bd2a..4ceb4d20e538a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -88,6 +88,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.PoliciesUtil; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.common.util.SecurityUtility; @@ -108,15 +109,41 @@ import org.awaitility.reflect.WhiteboxImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testng.ITest; +import org.testng.SkipException; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker-api") -public class BrokerServiceLookupTest extends ProducerConsumerBase { +public class BrokerServiceLookupTest extends ProducerConsumerBase implements ITest { private static final Logger log = LoggerFactory.getLogger(BrokerServiceLookupTest.class); + private String testName; + + @DataProvider + private static Object[] booleanValues() { + return new Object[]{ true, false }; + } + + @Factory(dataProvider = "booleanValues") + public BrokerServiceLookupTest(boolean useTestZookeeper) { + // when set to true, TestZKServer is used which is a real ZooKeeper implementation + this.useTestZookeeper = useTestZookeeper; + } + + @Override + public String getTestName() { + return testName; + } @BeforeMethod + public void applyTestName(Method method) { + testName = method.getName() + " with " + (useTestZookeeper ? "TestZKServer" : "MockZooKeeper"); + } + + @BeforeMethod(dependsOnMethods = "setTestMethodName") @Override protected void setup() throws Exception { conf.setDefaultNumberOfNamespaceBundles(1); @@ -125,10 +152,43 @@ protected void setup() throws Exception { producerBaseSetup(); } + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + switch (methodName) { + case "testMultipleBrokerDifferentClusterLookup" -> { + conf.setAuthenticationEnabled(true); + } + case "testWebserviceServiceTls" -> { + // broker1 with tls enabled + conf.setBrokerServicePortTls(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + conf.setTlsRequireTrustedClientCertOnConnect(true); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + conf.setNumExecutorThreadPoolSize(5); + // Not in use, and because TLS is not configured, it will fail to start + conf.setSystemTopicEnabled(false); + } + case "testSkipSplitBundleIfOnlyOneBroker" -> { + conf.setDefaultNumberOfNamespaceBundles(1); + conf.setLoadBalancerNamespaceBundleMaxTopics(1); + conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); + } + case "testPartitionedMetadataWithDeprecatedVersion" -> { + conf.setBrokerServicePortTls(Optional.empty()); + conf.setWebServicePortTls(Optional.empty()); + conf.setClientLibraryVersionCheckEnabled(true); + } + } + } + @AfterMethod(alwaysRun = true) @Override protected void cleanup() throws Exception { internalCleanup(); + testName = null; } /** @@ -214,9 +274,11 @@ public void testMultipleBrokerLookup() throws Exception { @Test public void testConcurrentWriteBrokerData() throws Exception { Map map = new ConcurrentHashMap<>(); + List boundaries = PoliciesUtil.getBundles(100).getBoundaries(); for (int i = 0; i < 100; i++) { - map.put("key"+ i, new NamespaceBundleStats()); + map.put("my-property/my-ns/" + boundaries.get(i), new NamespaceBundleStats()); } + BrokerService originalBrokerService = pulsar.getBrokerService(); BrokerService brokerService = mock(BrokerService.class); doReturn(brokerService).when(pulsar).getBrokerService(); doReturn(map).when(brokerService).getBundleStats(); @@ -247,6 +309,8 @@ public void testConcurrentWriteBrokerData() throws Exception { for (Future future : list) { future.get(); } + // allow proper shutdown so that resources aren't leaked + doReturn(originalBrokerService).when(pulsar).getBrokerService(); } /** @@ -294,12 +358,6 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { @Cleanup PulsarClient pulsarClient2 = PulsarClient.builder().serviceUrl(brokerServiceUrl.toString()).build(); - // enable authorization: so, broker can validate cluster and redirect if finds different cluster - pulsar.getConfiguration().setAuthorizationEnabled(true); - // restart broker with authorization enabled: it initialize AuthorizationService - stopBroker(); - startBroker(); - LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); loadManagerField.setAccessible(true); @@ -336,10 +394,6 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { consumer.acknowledgeCumulative(msg); consumer.close(); producer.close(); - - // disable authorization - pulsar.getConfiguration().setAuthorizationEnabled(false); - loadManager2 = null; } /** @@ -457,18 +511,6 @@ public void testWebserviceServiceTls() throws Exception { PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf2); PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); - // restart broker1 with tls enabled - conf.setBrokerServicePortTls(Optional.of(0)); - conf.setWebServicePortTls(Optional.of(0)); - conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); - conf.setTlsRequireTrustedClientCertOnConnect(true); - conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); - conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); - conf.setNumExecutorThreadPoolSize(5); - // Not in use, and because TLS is not configured, it will fail to start - conf.setSystemTopicEnabled(false); - stopBroker(); - startBroker(); pulsar.getLoadManager().get().writeLoadReportOnZookeeper(); pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); @@ -672,11 +714,6 @@ public void testModularLoadManagerSplitBundle() throws Exception { conf2.setLoadBalancerAutoUnloadSplitBundlesEnabled(true); conf2.setLoadBalancerNamespaceBundleMaxTopics(1); - // configure broker-1 with ModularLoadManager - stopBroker(); - conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); - startBroker(); - @Cleanup PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf2); PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); @@ -794,12 +831,6 @@ public void testSkipSplitBundleIfOnlyOneBroker() throws Exception { final String topicName1 = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp_"); final String topicName2 = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp_"); try { - // configure broker with ModularLoadManager. - stopBroker(); - conf.setDefaultNumberOfNamespaceBundles(1); - conf.setLoadBalancerNamespaceBundleMaxTopics(1); - conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); - startBroker(); final ModularLoadManagerWrapper modularLoadManagerWrapper = (ModularLoadManagerWrapper) pulsar.getLoadManager().get(); final ModularLoadManagerImpl modularLoadManager = @@ -952,12 +983,6 @@ public void testPartitionedMetadataWithDeprecatedVersion() throws Exception { admin.namespaces().createNamespace(property + "/" + cluster + "/" + namespace); admin.topics().createPartitionedTopic(dest.toString(), totalPartitions); - stopBroker(); - conf.setBrokerServicePortTls(Optional.empty()); - conf.setWebServicePortTls(Optional.empty()); - conf.setClientLibraryVersionCheckEnabled(true); - startBroker(); - URI brokerServiceUrl = new URI(pulsar.getSafeWebServiceAddress()); URL url = brokerServiceUrl.toURL(); @@ -1116,6 +1141,9 @@ public String authenticate(AuthenticationDataSource authData) throws Authenticat @Test public void testLookupConnectionNotCloseIfGetUnloadingExOrMetadataEx() throws Exception { + if (useTestZookeeper) { + throw new SkipException("This test case depends on MockZooKeeper"); + } String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); admin.topics().createNonPartitionedTopic(tpName); PulsarClientImpl pulsarClientImpl = (PulsarClientImpl) pulsarClient; @@ -1211,7 +1239,8 @@ private void makeAcquireBundleLockSuccess() throws Exception { } } - @Test(timeOut = 30000) + // TODO: This test is disabled since it's invalid. The test fails for both TestZKServer and MockZooKeeper. + @Test(timeOut = 30000, enabled = false) public void testLookupConnectionNotCloseIfFailedToAcquireOwnershipOfBundle() throws Exception { String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); admin.topics().createNonPartitionedTopic(tpName); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java index ef070250ca1aa..295120bf369e1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java @@ -40,7 +40,7 @@ public abstract class ProducerConsumerBase extends MockedPulsarServiceBaseTest { protected String methodName; @BeforeMethod(alwaysRun = true) - public void beforeMethod(Method m) throws Exception { + public void setTestMethodName(Method m) throws Exception { methodName = m.getName(); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 78e9980bc2187..21128c77477b4 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -71,9 +71,10 @@ @Slf4j public class ZKMetadataStore extends AbstractBatchedMetadataStore implements MetadataStoreExtended, MetadataStoreLifecycle { - public static final String ZK_SCHEME = "zk"; public static final String ZK_SCHEME_IDENTIFIER = "zk:"; + // ephemeralOwner value for persistent nodes + private static final long NOT_EPHEMERAL = 0L; private final String zkConnectString; private final String rootPath; @@ -128,12 +129,17 @@ public ZKMetadataStore(ZooKeeper zkc) { @VisibleForTesting @SneakyThrows public ZKMetadataStore(ZooKeeper zkc, MetadataStoreConfig config) { - super(config); + this(zkc, config, false); + } + @VisibleForTesting + @SneakyThrows + public ZKMetadataStore(ZooKeeper zkc, MetadataStoreConfig config, boolean isZkManaged) { + super(config); this.zkConnectString = null; this.rootPath = null; this.metadataStoreConfig = null; - this.isZkManaged = false; + this.isZkManaged = isZkManaged; this.zkc = zkc; this.sessionWatcher = new ZKSessionWatcher(zkc, this::receivedSessionEvent); zkc.addWatch("/", this::handleWatchEvent, AddWatchMode.PERSISTENT_RECURSIVE); @@ -477,7 +483,7 @@ public void close() throws Exception { private Stat getStat(String path, org.apache.zookeeper.data.Stat zkStat) { return new Stat(path, zkStat.getVersion(), zkStat.getCtime(), zkStat.getMtime(), - zkStat.getEphemeralOwner() != -1, + zkStat.getEphemeralOwner() != NOT_EPHEMERAL, zkStat.getEphemeralOwner() == zkc.getSessionId()); } diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java index 9a8e3ef5a2d4f..1e861bbf88cfd 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java @@ -207,6 +207,7 @@ public void tearDown() throws Exception { try { // cleanup for metrics. metadataStore.close(); + metadataStore = null; stopZKCluster(); } catch (Exception e) { LOG.error("Got Exception while trying to stop ZKCluster", e); @@ -236,6 +237,9 @@ public void tearDown() throws Exception { protected void startZKCluster() throws Exception { zkUtil.startCluster(); zkc = zkUtil.getZooKeeperClient(); + if (metadataStore != null) { + metadataStore.close(); + } metadataStore = new FaultInjectionMetadataStore( MetadataStoreExtended.create(zkUtil.getZooKeeperConnectString(), MetadataStoreConfig.builder().metadataStoreName("metastore-" + getClass().getSimpleName()).build())); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index dd256d32f8ddb..8f45e6e473a32 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -24,11 +24,18 @@ import io.etcd.jetcd.test.EtcdClusterExtension; import java.io.File; import java.net.URI; +import java.util.Arrays; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletionException; import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.MetadataStoreFactory; +import org.apache.pulsar.metadata.impl.MetadataStoreFactoryImpl; import org.apache.pulsar.tests.TestRetrySupport; import org.assertj.core.util.Files; import org.testng.annotations.AfterClass; @@ -36,19 +43,45 @@ import org.testng.annotations.DataProvider; public abstract class BaseMetadataStoreTest extends TestRetrySupport { + // to debug specific implementations, set the TEST_METADATA_PROVIDERS environment variable + // or temporarily hard code this value in the test class before running tests in the IDE + // supported values are ZooKeeper,Memory,RocksDB,Etcd,Oxia,MockZooKeeper + private static final String TEST_METADATA_PROVIDERS = System.getenv("TEST_METADATA_PROVIDERS"); + private static String originalMetadatastoreProvidersPropertyValue; protected TestZKServer zks; protected EtcdCluster etcdCluster; - + private String mockZkUrl; + // reference to keep the MockZooKeeper instance alive in MockZookeeperMetadataStoreProvider + private MetadataStore mockZkStoreRef; + private String zksConnectionString; + private String memoryConnectionString; + private String rocksdbConnectionString; + private File rocksDbDirectory; + private boolean running; @BeforeClass(alwaysRun = true) @Override public void setup() throws Exception { + running = true; incrementSetupNumber(); zks = new TestZKServer(); + zksConnectionString = zks.getConnectionString(); + memoryConnectionString = "memory:" + UUID.randomUUID(); + rocksDbDirectory = Files.newTemporaryFolder().getAbsoluteFile(); + rocksdbConnectionString = "rocksdb:" + rocksDbDirectory; + originalMetadatastoreProvidersPropertyValue = + System.getProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); + // register MockZooKeeperMetadataStoreProvider + System.setProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, + MockZooKeeperMetadataStoreProvider.class.getName()); + mockZkUrl = "mock-zk:" + UUID.randomUUID(); + // create a reference in MockZooKeeperMetadataStoreProvider to keep the MockZooKeeper instance alive + mockZkStoreRef = MetadataStoreFactory.create(mockZkUrl, MetadataStoreConfig.builder().build()); } @AfterClass(alwaysRun = true) @Override public void cleanup() throws Exception { + running = false; markCurrentSetupNumberCleaned(); if (zks != null) { zks.close(); @@ -59,38 +92,83 @@ public void cleanup() throws Exception { etcdCluster.close(); etcdCluster = null; } - } - private static String createTempFolder() { - File temp = Files.newTemporaryFolder(); - temp.deleteOnExit(); - return temp.getAbsolutePath(); + if (mockZkStoreRef != null) { + mockZkStoreRef.close(); + mockZkStoreRef = null; + mockZkUrl = null; + } + + if (rocksDbDirectory != null) { + Files.delete(rocksDbDirectory); + rocksDbDirectory = null; + } + + if (originalMetadatastoreProvidersPropertyValue != null) { + System.setProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, + originalMetadatastoreProvidersPropertyValue); + } else { + System.clearProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); + } } @DataProvider(name = "impl") public Object[][] implementations() { + // If the environment variable TEST_METADATA_PROVIDERS is set, only run the specified implementations + if (StringUtils.isNotBlank(TEST_METADATA_PROVIDERS)) { + return filterImplementations(TEST_METADATA_PROVIDERS.split(",")); + } + return allImplementations(); + } + + private Object[][] allImplementations() { // A Supplier must be used for the Zookeeper connection string parameter. The retried test run will // use the same arguments as the failed attempt. // The Zookeeper test server gets restarted by TestRetrySupport before the retry. // The new connection string won't be available to the test method unless a // Supplier lambda is used for providing the value. return new Object[][]{ - {"ZooKeeper", stringSupplier(() -> zks.getConnectionString())}, - {"Memory", stringSupplier(() -> "memory:" + UUID.randomUUID())}, - {"RocksDB", stringSupplier(() -> "rocksdb:" + createTempFolder())}, + {"ZooKeeper", stringSupplier(() -> zksConnectionString)}, + {"Memory", stringSupplier(() -> memoryConnectionString)}, + {"RocksDB", stringSupplier(() -> rocksdbConnectionString)}, {"Etcd", stringSupplier(() -> "etcd:" + getEtcdClusterConnectString())}, + {"MockZooKeeper", stringSupplier(() -> mockZkUrl)}, }; } @DataProvider(name = "distributedImpl") public Object[][] distributedImplementations() { - return new Object[][]{ - {"ZooKeeper", stringSupplier(() -> zks.getConnectionString())}, - {"Etcd", stringSupplier(() -> "etcd:" + getEtcdClusterConnectString())}, - }; + return filterImplementations("ZooKeeper", "Etcd"); + } + + @DataProvider(name = "zkImpls") + public Object[][] zkImplementations() { + return filterImplementations("ZooKeeper", "MockZooKeeper"); + } + + protected Object[][] filterImplementations(String... providers) { + Set providersSet = Set.of(providers); + return Arrays.stream(allImplementations()) + .filter(impl -> providersSet.contains(impl[0])) + .toArray(Object[][]::new); + } + + @DataProvider(name = "zkImpls") + public Object[][] zkImplementations() { + return filterImplementations("ZooKeeper", "MockZooKeeper"); + } + + protected Object[][] filterImplementations(String... providers) { + Set providersSet = Set.of(providers); + return Arrays.stream(allImplementations()) + .filter(impl -> providersSet.contains(impl[0])) + .toArray(Object[][]::new); } private synchronized String getEtcdClusterConnectString() { + if (!running) { + return null; + } if (etcdCluster == null) { etcdCluster = EtcdClusterExtension.builder().withClusterName("test").withNodes(1).withSsl(false).build() .cluster(); @@ -100,7 +178,26 @@ private synchronized String getEtcdClusterConnectString() { } public static Supplier stringSupplier(Supplier supplier) { - return supplier; + return new StringSupplier(supplier); + } + + // Implements toString() so that the test name is more descriptive + private static class StringSupplier implements Supplier { + private final Supplier supplier; + + public StringSupplier(Supplier supplier) { + this.supplier = supplier; + } + + @Override + public String get() { + return supplier.get(); + } + + @Override + public String toString() { + return get(); + } } protected String newKey() { @@ -146,4 +243,15 @@ public static boolean retryStrategically(Predicate predicate, int retryCou } return false; } + + /** + * Delete all the empty container nodes + * @param provider the metadata store provider + * @throws Exception + */ + protected void maybeTriggerDeletingEmptyContainers(String provider) throws Exception { + if ("ZooKeeper".equals(provider) && zks != null) { + zks.checkContainers(); + } + } } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java index c5b4012f0c8f9..bd068539cc549 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java @@ -70,6 +70,7 @@ public void testCounterDoesNotAutoReset(String provider, Supplier urlSup return; } String metadataUrl = urlSupplier.get(); + @Cleanup MetadataStoreExtended store1 = MetadataStoreExtended.create(metadataUrl, MetadataStoreConfig.builder().build()); CoordinationService cs1 = new CoordinationServiceImpl(store1); @@ -85,7 +86,7 @@ public void testCounterDoesNotAutoReset(String provider, Supplier urlSup store1.close(); // Delete all the empty container nodes - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); @Cleanup MetadataStoreExtended store2 = MetadataStoreExtended.create(metadataUrl, MetadataStoreConfig.builder().build()); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java index 591c0a23a9bad..14fe3d35325ff 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java @@ -69,7 +69,6 @@ import org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl; import org.awaitility.Awaitility; import org.mockito.stubbing.Answer; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -111,14 +110,7 @@ public void emptyCacheTest(String provider, Supplier urlSupplier) throws } } - @DataProvider(name = "zk") - public Object[][] zkimplementations() { - return new Object[][] { - { "ZooKeeper", stringSupplier(() -> zks.getConnectionString()) }, - }; - } - - @Test(dataProvider = "zk") + @Test(dataProvider = "zkImpls") public void crossStoreAddDelete(String provider, Supplier urlSupplier) throws Exception { @Cleanup MetadataStore store1 = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); @@ -183,7 +175,7 @@ private void multiStoreAddDelete(List> caches, int addOn, }); } - @Test(dataProvider = "zk") + @Test(dataProvider = "zkImpls") public void crossStoreUpdates(String provider, Supplier urlSupplier) throws Exception { String testName = "cross store updates"; @Cleanup @@ -493,11 +485,10 @@ public void readModifyUpdate(String provider, Supplier urlSupplier) thro * * @throws Exception */ - @Test - public void readModifyUpdateBadVersionRetry() throws Exception { - String url = zks.getConnectionString(); + @Test(dataProvider = "zkImpls") + public void readModifyUpdateBadVersionRetry(String provider, Supplier urlSupplier) throws Exception { @Cleanup - MetadataStore store = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); + MetadataStore store = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); MetadataCache cache = store.getMetadataCache(MyClass.class); @@ -511,7 +502,8 @@ public void readModifyUpdateBadVersionRetry() throws Exception { final var sourceStores = new ArrayList(); for (int i = 0; i < 20; i++) { - final var sourceStore = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); + final var sourceStore = + MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); sourceStores.add(sourceStore); final var objCache = sourceStore.getMetadataCache(MyClass.class); futures.add(objCache.readModifyUpdate(key1, v -> new MyClass(v.a, v.b + 1))); @@ -522,11 +514,10 @@ public void readModifyUpdateBadVersionRetry() throws Exception { } } - @Test - public void readModifyUpdateOrCreateRetryTimeout() throws Exception { - String url = zks.getConnectionString(); + @Test(dataProvider = "zkImpls") + public void readModifyUpdateOrCreateRetryTimeout(String provider, Supplier urlSupplier) throws Exception { @Cleanup - MetadataStore store = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); + MetadataStore store = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); MetadataCache cache = store.getMetadataCache(MyClass.class, MetadataCacheConfig.builder() .retryBackoff(new BackoffBuilder() diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java index a4c937611fd3f..30fbd9b836e92 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.metadata; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -70,18 +71,19 @@ public void sequentialKeys(String provider, Supplier urlSupplier) throws @Test(dataProvider = "impl") public void testPersistentOrEphemeralPut(String provider, Supplier urlSupplier) throws Exception { final String key1 = newKey(); + @Cleanup MetadataStoreExtended store = MetadataStoreExtended.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); store.put(key1, "value-1".getBytes(), Optional.empty(), EnumSet.noneOf(CreateOption.class)).join(); var value = store.get(key1).join().get(); assertEquals(value.getValue(), "value-1".getBytes()); - // assertFalse(value.getStat().isEphemeral()); // Todo : fix zkStat.getEphemeralOwner() != 0 from test zk + assertFalse(value.getStat().isEphemeral()); assertTrue(value.getStat().isFirstVersion()); var version = value.getStat().getVersion(); store.put(key1, "value-2".getBytes(), Optional.empty(), EnumSet.noneOf(CreateOption.class)).join(); value = store.get(key1).join().get(); assertEquals(value.getValue(), "value-2".getBytes()); - //assertFalse(value.getStat().isEphemeral()); // Todo : fix zkStat.getEphemeralOwner() != 0 from test zk + assertFalse(value.getStat().isEphemeral()); assertEquals(value.getStat().getVersion(), version + 1); final String key2 = newKey(); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java index 246661edc43ee..c6d8118dc5e8e 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.metadata; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -54,6 +55,7 @@ import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.assertj.core.util.Lists; import org.awaitility.Awaitility; +import org.testng.SkipException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -97,7 +99,7 @@ public void concurrentPutTest(String provider, Supplier urlSupplier) thr MetadataStoreConfig.builder().fsyncEnable(false).build()); String data = "data"; - String path = "/non-existing-key"; + String path = "/concurrentPutTest"; int concurrent = 50; List> futureList = new ArrayList<>(); for (int i = 0; i < concurrent; i++) { @@ -400,6 +402,10 @@ public void testDeleteRecursive(String provider, Supplier urlSupplier) t @Test(dataProvider = "impl") public void testDeleteUnusedDirectories(String provider, Supplier urlSupplier) throws Exception { + if (provider.equals("MockZooKeeper")) { + throw new SkipException("MockZooKeeper does not support deleteUnusedDirectories"); + } + @Cleanup MetadataStore store = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().fsyncEnable(false).build()); @@ -413,18 +419,18 @@ public void testDeleteUnusedDirectories(String provider, Supplier urlSup store.delete(prefix + "/a1/b1/c1", Optional.empty()).join(); store.delete(prefix + "/a1/b1/c2", Optional.empty()).join(); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); assertFalse(store.exists(prefix + "/a1/b1").join()); store.delete(prefix + "/a1/b2/c1", Optional.empty()).join(); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); assertFalse(store.exists(prefix + "/a1/b2").join()); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); assertFalse(store.exists(prefix + "/a1").join()); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); assertFalse(store.exists(prefix).join()); } @@ -602,21 +608,25 @@ public void testGetChildren(String provider, Supplier urlSupplier) throw store.put("/b/c/b/1", "value1".getBytes(StandardCharsets.UTF_8), Optional.empty()).join(); List subPaths = store.getChildren("/").get(); - Set expectedSet = "ZooKeeper".equals(provider) ? Set.of("a", "b", "zookeeper") : Set.of("a", "b"); + Set ignoredRootPaths = Set.of("zookeeper"); + Set expectedSet = Set.of("a", "b"); for (String subPath : subPaths) { - assertTrue(expectedSet.contains(subPath)); + if (ignoredRootPaths.contains(subPath)) { + continue; + } + assertThat(expectedSet).contains(subPath); } List subPaths2 = store.getChildren("/a").get(); Set expectedSet2 = Set.of("a-1", "a-2"); for (String subPath : subPaths2) { - assertTrue(expectedSet2.contains(subPath)); + assertThat(expectedSet2).contains(subPath); } List subPaths3 = store.getChildren("/b").get(); Set expectedSet3 = Set.of("c"); for (String subPath : subPaths3) { - assertTrue(expectedSet3.contains(subPath)); + assertThat(expectedSet3).contains(subPath); } } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MockZooKeeperMetadataStoreProvider.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MockZooKeeperMetadataStoreProvider.java new file mode 100644 index 0000000000000..994a97c2b1053 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MockZooKeeperMetadataStoreProvider.java @@ -0,0 +1,49 @@ +/* + * 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.metadata; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreProvider; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.zookeeper.MockZooKeeper; +import org.apache.zookeeper.MockZooKeeperSession; + +public class MockZooKeeperMetadataStoreProvider implements MetadataStoreProvider { + private static final String MOCK_ZK_SCHEME = "mock-zk"; + private static final ConcurrentMap mockZooKeepers = new ConcurrentHashMap<>(); + + @Override + public String urlScheme() { + return MOCK_ZK_SCHEME; + } + + @Override + public MetadataStore create(String metadataURL, MetadataStoreConfig metadataStoreConfig, + boolean enableSessionWatcher) throws MetadataStoreException { + MockZooKeeper mockZooKeeper = mockZooKeepers.computeIfAbsent(metadataURL, + k -> MockZooKeeper.newInstance().registerCloseable(() -> mockZooKeepers.remove(k))); + MockZooKeeperSession mockZooKeeperSession = MockZooKeeperSession.newInstance(mockZooKeeper, true); + ZKMetadataStore zkMetadataStore = new ZKMetadataStore(mockZooKeeperSession, metadataStoreConfig, true); + return zkMetadataStore; + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java index b64cc964a999c..bf67d0218b0b7 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java @@ -373,7 +373,7 @@ public void testWithSeveralIncompletePaths(String provider, Supplier url assertEquals(ledgersReadAsync, ids, "Comparing LedgersIds read asynchronously"); } - @Test(timeOut = 30000, dataProvider = "impl") + @Test(timeOut = 60000, dataProvider = "impl") public void checkConcurrentModifications(String provider, Supplier urlSupplier) throws Throwable { @Cleanup MetadataStoreExtended store = @@ -406,14 +406,16 @@ public void checkConcurrentModifications(String provider, Supplier urlSu ExecutorService executor = Executors.newCachedThreadPool(); final ConcurrentSkipListSet createdLedgers = new ConcurrentSkipListSet<>(); for (int i = 0; i < numWriters; ++i) { + int writerIndex = i; Future f = executor.submit(() -> { @Cleanup LedgerManager writerLM = new PulsarLedgerManager(store, ledgersRoot); Random writerRNG = new Random(rng.nextLong()); - + log.info("Writer {} waiting", writerIndex); latch.await(); - + log.info("Writer {} started", writerIndex); while (MathUtils.elapsedNanos(start) < runtime) { + log.info("Writer {} writing", writerIndex); long candidate = 0; do { candidate = Math.abs(writerRNG.nextLong()); @@ -425,18 +427,22 @@ public void checkConcurrentModifications(String provider, Supplier urlSu createLedger(writerLM, candidate); removeLedger(writerLM, candidate); } + log.info("Writer {} finished", writerIndex); return null; }); futures.add(f); } for (int i = 0; i < numCheckers; ++i) { + int checkerIndex = i; Future f = executor.submit(() -> { @Cleanup LedgerManager checkerLM = new PulsarLedgerManager(store, ledgersRoot); + log.info("Checker {} waiting", checkerIndex); latch.await(); - + log.info("Checker {} started", checkerIndex); while (MathUtils.elapsedNanos(start) < runtime) { + log.info("Checker {} checking", checkerIndex); LedgerRangeIterator lri = checkerLM.getLedgerRanges(0); Set returnedIds = ledgerRangeToSet(lri); for (long id : mustExist) { @@ -448,15 +454,19 @@ public void checkConcurrentModifications(String provider, Supplier urlSu assertTrue(ledgersReadAsync.contains(id)); } } + log.info("Checker {} finished", checkerIndex); return null; }); futures.add(f); } latch.countDown(); + log.info("Waiting for futures"); for (Future f : futures) { + log.info("Waiting for future"); f.get(); } + log.info("Completed"); executor.shutdownNow(); } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java index 0e9c781fb9143..ac73491a81c65 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java @@ -300,10 +300,10 @@ public void testMarkingAsReplicated(String provider, Supplier urlSupplie assertEquals(l, lB.get(), "Should be the ledger I marked"); } - - @Test(timeOut = 10000) - public void testZkMetasStoreMarkReplicatedDeleteEmptyParentNodes() throws Exception { - methodSetup(stringSupplier(() -> zks.getConnectionString())); + @Test(dataProvider = "zkImpls", timeOut = 10000) + public void testZkMetasStoreMarkReplicatedDeleteEmptyParentNodes(String provider, Supplier urlSupplier) + throws Exception { + methodSetup(urlSupplier); String missingReplica = "localhost:3181"; diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerIdGeneratorTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerIdGeneratorTest.java index 73d5f451c1ff1..da3fd7f7bd443 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerIdGeneratorTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerIdGeneratorTest.java @@ -242,7 +242,7 @@ public void testEnsureCounterIsNotResetWithContainerNodes(String provider, Suppl l1.await(); log.info("res1 : {}", res1); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); CountDownLatch l2 = new CountDownLatch(1); AtomicLong res2 = new AtomicLong(); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java index c0159be4303bc..34e860aa57845 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java @@ -20,6 +20,10 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; +import java.util.EnumSet; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; import lombok.Cleanup; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataStore; @@ -31,26 +35,25 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.util.EnumSet; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; public class MetadataStoreFactoryImplTest { - - private static Object originalProperty; + private static String originalMetadatastoreProvidersPropertyValue; @BeforeClass public void setMetadataStoreProperty() { - originalProperty = System.getProperties().get(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); + originalMetadatastoreProvidersPropertyValue = + System.getProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); System.setProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, MyMetadataStoreProvider.class.getName()); } @AfterClass public void resetMetadataStoreProperty() { - if (originalProperty != null) { - System.getProperties().put(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, originalProperty); + if (originalMetadatastoreProvidersPropertyValue != null) { + System.setProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, + originalMetadatastoreProvidersPropertyValue); + } else { + System.clearProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); } } diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index f32036e53f001..e124699ee1383 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -19,34 +19,32 @@ package org.apache.zookeeper; import com.google.common.collect.HashMultimap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.MoreExecutors; import io.netty.util.concurrent.DefaultThreadFactory; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.TreeMap; -import java.util.TreeSet; +import java.util.concurrent.Callable; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; import lombok.AllArgsConstructor; -import lombok.Data; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.zookeeper.AsyncCallback.Children2Callback; import org.apache.zookeeper.AsyncCallback.ChildrenCallback; import org.apache.zookeeper.AsyncCallback.DataCallback; @@ -57,6 +55,8 @@ import org.apache.zookeeper.Watcher.Event.KeeperState; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.proto.DeleteRequest; +import org.apache.zookeeper.proto.SetDataRequest; import org.objenesis.Objenesis; import org.objenesis.ObjenesisStd; import org.objenesis.instantiator.ObjectInstantiator; @@ -64,33 +64,79 @@ import org.slf4j.LoggerFactory; public class MockZooKeeper extends ZooKeeper { - @Data + // ephemeralOwner value for persistent nodes + private static final long NOT_EPHEMERAL = 0L; + private static final String ROOT_PATH = "/"; + @AllArgsConstructor private static class MockZNode { byte[] content; int version; long ephemeralOwner; + long creationTimestamp; + long modificationTimestamp; + List children; static MockZNode of(byte[] content, int version, long ephemeralOwner) { - return new MockZNode(content, version, ephemeralOwner); + return new MockZNode(content, version, ephemeralOwner, System.currentTimeMillis(), + System.currentTimeMillis(), new ArrayList<>()); + } + + public void updateVersion() { + version++; + modificationTimestamp = System.currentTimeMillis(); + } + + public void updateData(byte[] data) { + content = data; + updateVersion(); + } + + public Stat getStat() { + return applyToStat(new Stat()); + } + + public Stat applyToStat(Stat stat) { + stat.setCtime(creationTimestamp); + stat.setMtime(modificationTimestamp); + stat.setVersion(version); + stat.setEphemeralOwner(ephemeralOwner); + return stat; + } + + public int getVersion() { + return version; + } + + public byte[] getContent() { + return content; + } + + public long getEphemeralOwner() { + return ephemeralOwner; + } + + public List getChildren() { + return children; } } private TreeMap tree; - private SetMultimap watchers; - private volatile boolean stopped; + private SetMultimap watchers; + private AtomicBoolean stopped; private AtomicReference alwaysFail; private CopyOnWriteArrayList failures; private ExecutorService executor; - private Watcher sessionWatcher; - private long sessionId = 0L; + private volatile Watcher sessionWatcher; + private long sessionId = 1L; private int readOpDelayMs; - private ReentrantLock mutex; - private AtomicLong sequentialIdGenerator; - private ThreadLocal epheralOwnerThreadLocal; + private ThreadLocal overriddenSessionIdThreadLocal; + private ThreadLocal inExecutorThreadLocal; + private int referenceCount; + private List closeables; //see details of Objenesis caching - http://objenesis.org/details.html //see supported jvms - https://github.com/easymock/objenesis/blob/master/SupportedJVMs.md @@ -110,41 +156,21 @@ private static class Failure { } } - @Data - @AllArgsConstructor - private static class PersistentWatcher { - final String path; - final Watcher watcher; - final AddWatchMode mode; + private record PersistentWatcher(String path, Watcher watcher, AddWatchMode mode, long sessionId) { } - private List persistentWatchers; - - public static MockZooKeeper newInstance() { - return newInstance(null); + private record NodeWatcher(Watcher watcher, long sessionId) { } - public static MockZooKeeper newInstance(ExecutorService executor) { - return newInstance(executor, -1); - } - - public static MockZooKeeper newInstanceForGlobalZK(ExecutorService executor) { - return newInstanceForGlobalZK(executor, -1); - } + private List persistentWatchers; - public static MockZooKeeper newInstanceForGlobalZK(ExecutorService executor, int readOpDelayMs) { - try { - return createMockZooKeeperInstance(executor, readOpDelayMs); - } catch (RuntimeException e) { - throw e; - } catch (Exception e) { - throw new IllegalStateException("Cannot create object", e); - } + public static MockZooKeeper newInstance() { + return newInstance(-1); } - public static MockZooKeeper newInstance(ExecutorService executor, int readOpDelayMs) { + public static MockZooKeeper newInstance(int readOpDelayMs) { try { - return createMockZooKeeperInstance(executor, readOpDelayMs); + return createMockZooKeeperInstance(readOpDelayMs); } catch (RuntimeException e) { throw e; } catch (Exception e) { @@ -152,29 +178,25 @@ public static MockZooKeeper newInstance(ExecutorService executor, int readOpDela } } - private static MockZooKeeper createMockZooKeeperInstance(ExecutorService executor, int readOpDelayMs) { + private static MockZooKeeper createMockZooKeeperInstance(int readOpDelayMs) { ObjectInstantiator mockZooKeeperInstantiator = objenesis.getInstantiatorOf(MockZooKeeper.class); MockZooKeeper zk = mockZooKeeperInstantiator.newInstance(); - zk.epheralOwnerThreadLocal = new ThreadLocal<>(); - zk.init(executor); + zk.overriddenSessionIdThreadLocal = new ThreadLocal<>(); + zk.inExecutorThreadLocal = ThreadLocal.withInitial(() -> false); + zk.init(); zk.readOpDelayMs = readOpDelayMs; - zk.mutex = new ReentrantLock(); - zk.lockInstance = ThreadLocal.withInitial(zk::createLock); zk.sequentialIdGenerator = new AtomicLong(); + zk.closeables = new ArrayList<>(); return zk; } - private void init(ExecutorService executor) { + private void init() { tree = Maps.newTreeMap(); - if (executor != null) { - this.executor = executor; - } else { - this.executor = Executors.newFixedThreadPool(1, new DefaultThreadFactory("mock-zookeeper")); - } - SetMultimap w = HashMultimap.create(); - watchers = Multimaps.synchronizedSetMultimap(w); - stopped = false; + tree.put(ROOT_PATH, MockZNode.of(new byte[0], 0, NOT_EPHEMERAL)); + this.executor = Executors.newSingleThreadExecutor(new DefaultThreadFactory("mock-zookeeper")); + watchers = HashMultimap.create(); + stopped = new AtomicBoolean(false); alwaysFail = new AtomicReference<>(KeeperException.Code.OK); failures = new CopyOnWriteArrayList<>(); persistentWatchers = new ArrayList<>(); @@ -197,101 +219,143 @@ public States getState() { return States.CONNECTED; } + @Override + public void register(Watcher watcher) { + sessionWatcher = watcher; + } - @Slf4j - private static class SingleAcquireAndReleaseLock { - private final AtomicBoolean acquired = new AtomicBoolean(false); - private final Lock lock; + @Override + public String create(String path, byte[] data, List acl, CreateMode createMode) + throws KeeperException, InterruptedException { + return runInExecutorReturningValue(() -> internalCreate(path, data, createMode)); + } - SingleAcquireAndReleaseLock(Lock lock) { - this.lock = lock; + private T runInExecutorReturningValue(Callable task) + throws InterruptedException, KeeperException { + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); } - - public void lock() { - if (acquired.compareAndSet(false, true)) { - lock.lock(); - } else { - throw new IllegalStateException("Lock was already acquired!"); + if (inExecutorThreadLocal.get()) { + try { + return task.call(); + } catch (Exception e) { + if (e instanceof KeeperException ke) { + throw ke; + } + if (e instanceof InterruptedException ie) { + throw ie; + } + log.error("Unexpected exception", e); + throw new KeeperException.SystemErrorException(); } } - - public void unlockIfNeeded() { - if (acquired.compareAndSet(true, false)) { - lock.unlock(); + try { + long currentSessionId = getSessionId(); + return executor.submit(() -> { + inExecutorThreadLocal.set(true); + overrideSessionId(currentSessionId); + try { + return task.call(); + } finally { + removeSessionIdOverride(); + inExecutorThreadLocal.set(false); + } + }).get(); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof KeeperException ke) { + throw ke; + } + if (cause instanceof InterruptedException ie) { + throw ie; } + log.error("Unexpected exception", e); + throw new KeeperException.SystemErrorException(); } } - private ThreadLocal lockInstance; - - private SingleAcquireAndReleaseLock createLock() { - return new SingleAcquireAndReleaseLock(mutex); - } - - private void lock() { - lockInstance.get().lock(); - } - - private void unlockIfLocked() { - lockInstance.get().unlockIfNeeded(); + private void runInExecutorAsync(Runnable runnable) { + if (isStopped()) { + throw new RejectedExecutionException("MockZooKeeper is stopped"); + } + if (inExecutorThreadLocal.get()) { + try { + runnable.run(); + } catch (Throwable t) { + log.error("Unexpected exception", t); + } + return; + } + long currentSessionId = getSessionId(); + executor.submit(() -> { + try { + inExecutorThreadLocal.set(true); + overrideSessionId(currentSessionId); + try { + runnable.run(); + } finally { + removeSessionIdOverride(); + inExecutorThreadLocal.set(false); + } + } catch (Throwable t) { + log.error("Unexpected exception", t); + } + }); } - @Override - public void register(Watcher watcher) { - lock(); - sessionWatcher = watcher; - unlockIfLocked(); + private void runInExecutorSync(Runnable runnable) { + try { + runInExecutorReturningValue(() -> { + runnable.run(); + return null; + }); + } catch (Exception e) { + log.error("Unexpected error", e); + } } - @Override - public String create(String path, byte[] data, List acl, CreateMode createMode) - throws KeeperException, InterruptedException { + private String internalCreate(String path, byte[] data, CreateMode createMode) throws KeeperException { final Set toNotifyCreate = Sets.newHashSet(); final Set toNotifyParent = Sets.newHashSet(); - final String parent = path.substring(0, path.lastIndexOf("/")); - - lock(); - try { - + final String parent = getParentName(path); - maybeThrowProgrammedFailure(Op.CREATE, path); + maybeThrowProgrammedFailure(Op.CREATE, path); - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } - - if (tree.containsKey(path)) { - throw new KeeperException.NodeExistsException(path); - } + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); + } - if (!parent.isEmpty() && !tree.containsKey(parent)) { - throw new KeeperException.NoNodeException(); - } + if (tree.containsKey(path)) { + throw new KeeperException.NodeExistsException(path); + } - if (createMode.isSequential()) { - MockZNode parentNode = tree.get(parent); - int parentVersion = tree.get(parent).getVersion(); - path = path + parentVersion; + MockZNode parentNode = tree.get(parent); - // Update parent version - tree.put(parent, - MockZNode.of(parentNode.getContent(), parentVersion + 1, parentNode.getEphemeralOwner())); - } + if (parentNode == null) { + throw new KeeperException.NoNodeException(parent); + } - tree.put(path, MockZNode.of(data, 0, createMode.isEphemeral() ? getEphemeralOwner() : -1L)); + if (createMode.isSequential()) { + int parentVersion = parentNode.getVersion(); + path = path + parentVersion; + parentNode.updateVersion(); + } - toNotifyCreate.addAll(watchers.get(path)); + parentNode.getChildren().add(getNodeName(path)); + tree.put(path, createMockZNode(data, createMode)); - if (!parent.isEmpty()) { - toNotifyParent.addAll(watchers.get(parent)); - } - watchers.removeAll(path); - } finally { - unlockIfLocked(); + toNotifyCreate.addAll(getWatchers(path)); + if (!ROOT_PATH.equals(parent)) { + toNotifyParent.addAll(getWatchers(parent)); } + watchers.removeAll(path); final String finalPath = path; executor.execute(() -> { + if (isStopped()) { + return; + } + triggerPersistentWatches(finalPath, parent, EventType.NodeCreated); toNotifyCreate.forEach( @@ -309,43 +373,62 @@ public String create(String path, byte[] data, List acl, CreateMode createM return path; } - protected long getEphemeralOwner() { - Long epheralOwner = epheralOwnerThreadLocal.get(); - if (epheralOwner != null) { - return epheralOwner; + private static String getParentName(String path) { + String parentName = path.substring(0, path.lastIndexOf('/')); + return parentName.length() > 0 ? parentName : "/"; + } + + private static String getNodeName(String path) { + return path.substring(path.lastIndexOf('/') + 1); + } + + private Collection getWatchers(String path) { + Set nodeWatchers = watchers.get(path); + if (nodeWatchers != null) { + return nodeWatchers.stream().map(NodeWatcher::watcher).toList(); + } else { + return Collections.emptyList(); } - return getSessionId(); } - public void overrideEpheralOwner(long epheralOwner) { - epheralOwnerThreadLocal.set(epheralOwner); + @Override + public long getSessionId() { + Long overriddenSessionId = overriddenSessionIdThreadLocal.get(); + if (overriddenSessionId != null) { + return overriddenSessionId; + } + return sessionId; + } + + public void overrideSessionId(long sessionId) { + overriddenSessionIdThreadLocal.set(sessionId); } - public void removeEpheralOwnerOverride() { - epheralOwnerThreadLocal.remove(); + public void removeSessionIdOverride() { + overriddenSessionIdThreadLocal.remove(); } @Override public void create(final String path, final byte[] data, final List acl, CreateMode createMode, final StringCallback cb, final Object ctx) { - - - executor.execute(() -> { + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); + return; + } + runInExecutorAsync(() -> { try { - lock(); - - if (stopped) { + if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } final Set toNotifyCreate = Sets.newHashSet(); - toNotifyCreate.addAll(watchers.get(path)); + toNotifyCreate.addAll(getWatchers(path)); final Set toNotifyParent = Sets.newHashSet(); - final String parent = path.substring(0, path.lastIndexOf("/")); - if (!parent.isEmpty()) { - toNotifyParent.addAll(watchers.get(parent)); + final String parent = getParentName(path); + if (!ROOT_PATH.equals(parent)) { + toNotifyParent.addAll(getWatchers(parent)); } final String name; @@ -357,355 +440,247 @@ public void create(final String path, final byte[] data, final List acl, Cr Optional failure = programmedFailure(Op.CREATE, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null); - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); } else if (tree.containsKey(path)) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NODEEXISTS.intValue(), path, ctx, null); - } else if (!parent.isEmpty() && !tree.containsKey(parent)) { - unlockIfLocked(); - toNotifyParent.forEach(watcher -> watcher - .process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, - parent))); + } else if (!tree.containsKey(parent)) { + runNotifications(() -> { + toNotifyParent.forEach(watcher -> watcher + .process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, + parent))); + }); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); } else { - tree.put(name, MockZNode.of(data, 0, - createMode != null && createMode.isEphemeral() ? getEphemeralOwner() : -1L)); + tree.get(parent).getChildren().add(getNodeName(name)); + tree.put(name, createMockZNode(data, createMode)); watchers.removeAll(name); - unlockIfLocked(); cb.processResult(0, path, ctx, name); - - triggerPersistentWatches(path, parent, EventType.NodeCreated); - - toNotifyCreate.forEach( - watcher -> watcher.process( - new WatchedEvent(EventType.NodeCreated, - KeeperState.SyncConnected, - name))); - toNotifyParent.forEach( - watcher -> watcher.process( - new WatchedEvent(EventType.NodeChildrenChanged, - KeeperState.SyncConnected, - parent))); + runNotifications(() -> { + triggerPersistentWatches(path, parent, EventType.NodeCreated); + + toNotifyCreate.forEach( + watcher -> watcher.process( + new WatchedEvent(EventType.NodeCreated, + KeeperState.SyncConnected, + name))); + toNotifyParent.forEach( + watcher -> watcher.process( + new WatchedEvent(EventType.NodeChildrenChanged, + KeeperState.SyncConnected, + parent))); + }); } } catch (Throwable ex) { log.error("create path : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); - } finally { - unlockIfLocked(); } }); + } + + public void runNotifications(Runnable runnable) { + executor.execute(() -> { + if (isStopped()) { + return; + } + runnable.run(); + }); + } + private boolean isStopped() { + return stopped.get(); + } + + private MockZNode createMockZNode(byte[] data, CreateMode createMode) { + return MockZNode.of(data, 0, + createMode != null && createMode.isEphemeral() ? getSessionId() : NOT_EPHEMERAL); } @Override - public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException { - lock(); - try { - maybeThrowProgrammedFailure(Op.GET, path); - MockZNode value = tree.get(path); - if (value == null) { - throw new KeeperException.NoNodeException(path); - } else { - if (watcher != null) { - watchers.put(path, watcher); - } - if (stat != null) { - applyToStat(value, stat); - } - return value.getContent(); + public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException, InterruptedException { + return runInExecutorReturningValue(() -> internalGetData(path, watcher, stat)); + } + + private byte[] internalGetData(String path, Watcher watcher, Stat stat) throws KeeperException { + maybeThrowProgrammedFailure(Op.GET, path); + MockZNode value = tree.get(path); + if (value == null) { + throw new KeeperException.NoNodeException(path); + } else { + if (watcher != null) { + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } - } finally { - unlockIfLocked(); + if (stat != null) { + value.applyToStat(stat); + } + return value.getContent(); } } @Override public void getData(final String path, boolean watch, final DataCallback cb, final Object ctx) { - executor.execute(() -> { - try { - checkReadOpDelay(); - Optional failure = programmedFailure(Op.GET, path); - if (failure.isPresent()) { - cb.processResult(failure.get().intValue(), path, ctx, null, null); - return; - } else if (stopped) { - cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); - return; - } - - MockZNode value; - lock(); - try { - value = tree.get(path); - } finally { - unlockIfLocked(); - } - - if (value == null) { - cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null, null); - } else { - cb.processResult(0, path, ctx, value.getContent(), createStatForZNode(value)); - } - } catch (Throwable ex) { - log.error("get data : {} error", path, ex); - cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); - } - }); + getData(path, null, cb, ctx); } @Override public void getData(final String path, final Watcher watcher, final DataCallback cb, final Object ctx) { - executor.execute(() -> { + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); + return; + } + runInExecutorAsync(() -> { checkReadOpDelay(); try { - lock(); Optional failure = programmedFailure(Op.GET, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null, null); return; - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); return; } MockZNode value = tree.get(path); if (value == null) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null, null); } else { if (watcher != null) { - watchers.put(path, watcher); + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } - - Stat stat = createStatForZNode(value); - unlockIfLocked(); + Stat stat = value.getStat(); cb.processResult(0, path, ctx, value.getContent(), stat); } } catch (Throwable ex) { log.error("get data : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); - } finally { - unlockIfLocked(); } }); } @Override public void getChildren(final String path, final Watcher watcher, final ChildrenCallback cb, final Object ctx) { - executor.execute(() -> { - List children = Lists.newArrayList(); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); + return; + } + runInExecutorAsync(() -> { try { - lock(); Optional failure = programmedFailure(Op.GET_CHILDREN, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null); return; - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } if (!tree.containsKey(path)) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); return; } - for (String item : tree.tailMap(path).keySet()) { - if (!item.startsWith(path)) { - break; - } else { - if (path.length() >= item.length()) { - continue; - } - - String child = item.substring(path.length() + 1); - if (item.charAt(path.length()) == '/' && !child.contains("/")) { - children.add(child); - } - } - } - + List children = findFirstLevelChildren(path); if (watcher != null) { - watchers.put(path, watcher); + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } cb.processResult(0, path, ctx, children); } catch (Throwable ex) { log.error("get children : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); - } finally { - unlockIfLocked(); } - }); } @Override - public List getChildren(String path, Watcher watcher) throws KeeperException { - lock(); - try { - maybeThrowProgrammedFailure(Op.GET_CHILDREN, path); - - if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(); - } - - String firstKey = path.equals("/") ? path : path + "/"; - String lastKey = path.equals("/") ? "0" : path + "0"; // '0' is lexicographically just after '/' + public List getChildren(String path, Watcher watcher) throws KeeperException, InterruptedException { + return runInExecutorReturningValue(() -> internalGetChildren(path, watcher)); + } - Set children = new TreeSet<>(); - tree.subMap(firstKey, false, lastKey, false).forEach((key, value) -> { - String relativePath = key.replace(firstKey, ""); + private List internalGetChildren(String path, Watcher watcher) throws KeeperException { + maybeThrowProgrammedFailure(Op.GET_CHILDREN, path); - // Only return first-level children - String child = relativePath.split("/", 2)[0]; - children.add(child); - }); - - if (watcher != null) { - watchers.put(path, watcher); - } + if (!tree.containsKey(path)) { + throw new KeeperException.NoNodeException(path); + } - return new ArrayList<>(children); - } finally { - unlockIfLocked(); + if (watcher != null) { + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } + + return findFirstLevelChildren(path); } @Override public List getChildren(String path, boolean watch) throws KeeperException, InterruptedException { - lock(); - try { - maybeThrowProgrammedFailure(Op.GET_CHILDREN, path); - - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } else if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(); - } - - String firstKey = path.equals("/") ? path : path + "/"; - String lastKey = path.equals("/") ? "0" : path + "0"; // '0' is lexicographically just after '/' - - Set children = new TreeSet<>(); - tree.subMap(firstKey, false, lastKey, false).forEach((key, value) -> { - String relativePath = key.replace(firstKey, ""); - - // Only return first-level children - String child = relativePath.split("/", 2)[0]; - children.add(child); - }); - - return new ArrayList<>(children); - } finally { - unlockIfLocked(); - } + return getChildren(path, null); } @Override public void getChildren(final String path, boolean watcher, final Children2Callback cb, final Object ctx) { - executor.execute(() -> { - Set children = new TreeSet<>(); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); + return; + } + runInExecutorAsync(() -> { try { - lock(); + MockZNode mockZNode = tree.get(path); + Stat stat = mockZNode != null ? mockZNode.getStat() : null; Optional failure = programmedFailure(Op.GET_CHILDREN, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null, null); return; - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); return; - } else if (!tree.containsKey(path)) { - unlockIfLocked(); + } else if (mockZNode == null) { cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null, null); return; } - String firstKey = path.equals("/") ? path : path + "/"; - String lastKey = path.equals("/") ? "0" : path + "0"; // '0' is lexicographically just after '/' - - tree.subMap(firstKey, false, lastKey, false).forEach((key, value) -> { - String relativePath = key.replace(firstKey, ""); - - // Only return first-level children - String child = relativePath.split("/", 2)[0]; - children.add(child); - }); - cb.processResult(0, path, ctx, new ArrayList<>(children), new Stat()); + List children = findFirstLevelChildren(path); + cb.processResult(0, path, ctx, children, stat); } catch (Throwable ex) { log.error("get children : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); - } finally { - unlockIfLocked(); } }); + } + private List findFirstLevelChildren(String path) { + return new ArrayList<>(tree.get(path).getChildren()); + } + + private boolean hasChildren(String path) { + return !tree.get(path).getChildren().isEmpty(); } @Override public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { - lock(); - try { - maybeThrowProgrammedFailure(Op.EXISTS, path); + return runInExecutorReturningValue(() -> internalGetStat(path, null)); + } - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } + private Stat internalGetStat(String path, Watcher watcher) throws KeeperException { + maybeThrowProgrammedFailure(Op.EXISTS, path); - if (tree.containsKey(path)) { - return createStatForZNode(tree.get(path)); - } else { - return null; - } - } finally { - unlockIfLocked(); + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); } - } - private static Stat createStatForZNode(MockZNode zNode) { - return applyToStat(zNode, new Stat()); - } + if (watcher != null) { + watchers.put(path, new NodeWatcher(watcher, getSessionId())); + } - private static Stat applyToStat(MockZNode zNode, Stat stat) { - stat.setVersion(zNode.getVersion()); - if (zNode.getEphemeralOwner() != -1L) { - stat.setEphemeralOwner(zNode.getEphemeralOwner()); + if (tree.containsKey(path)) { + return tree.get(path).getStat(); + } else { + return null; } - return stat; } @Override public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException { - lock(); - try { - maybeThrowProgrammedFailure(Op.EXISTS, path); - - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } - - if (watcher != null) { - watchers.put(path, watcher); - } - - if (tree.containsKey(path)) { - return createStatForZNode(tree.get(path)); - } else { - return null; - } - } finally { - unlockIfLocked(); - } + return runInExecutorReturningValue(() -> internalGetStat(path, watcher)); } @Override @@ -715,160 +690,149 @@ public void exists(String path, boolean watch, StatCallback cb, Object ctx) { @Override public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { - executor.execute(() -> { + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); + return; + } + runInExecutorAsync(() -> { try { - lock(); Optional failure = programmedFailure(Op.EXISTS, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null); return; - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } if (watcher != null) { - watchers.put(path, watcher); + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } - if (tree.containsKey(path)) { - unlockIfLocked(); - cb.processResult(0, path, ctx, new Stat()); + MockZNode mockZNode = tree.get(path); + if (mockZNode != null) { + Stat stat = mockZNode.getStat(); + cb.processResult(0, path, ctx, stat); } else { - unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); } } catch (Throwable ex) { log.error("exist : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); - } finally { - unlockIfLocked(); } }); } @Override public void sync(String path, VoidCallback cb, Object ctx) { - executor.execute(() -> { + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); + return; + } + runInExecutorAsync(() -> { Optional failure = programmedFailure(Op.SYNC, path); if (failure.isPresent()) { cb.processResult(failure.get().intValue(), path, ctx); return; - } else if (stopped) { + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); return; } - cb.processResult(0, path, ctx); }); - } @Override public Stat setData(final String path, byte[] data, int version) throws KeeperException, InterruptedException { - final Set toNotify = Sets.newHashSet(); - MockZNode newZNode; - - lock(); - try { - maybeThrowProgrammedFailure(Op.SET, path); - - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } + return runInExecutorReturningValue(() -> internalSetData(path, data, version)); + } - if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(); - } + private Stat internalSetData(String path, byte[] data, int version) throws KeeperException { + final Set toNotify = Sets.newHashSet(); + maybeThrowProgrammedFailure(Op.SET, path); - MockZNode mockZNode = tree.get(path); - int currentVersion = mockZNode.getVersion(); + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); + } - // Check version - if (version != -1 && version != currentVersion) { - throw new KeeperException.BadVersionException(path); - } + if (!tree.containsKey(path)) { + throw new KeeperException.NoNodeException(path); + } - log.debug("[{}] Updating -- current version: {}", path, currentVersion); - newZNode = MockZNode.of(data, currentVersion + 1, mockZNode.getEphemeralOwner()); - tree.put(path, newZNode); + MockZNode mockZNode = tree.get(path); + int currentVersion = mockZNode.getVersion(); - toNotify.addAll(watchers.get(path)); - watchers.removeAll(path); - } finally { - unlockIfLocked(); + // Check version + if (version != -1 && version != currentVersion) { + throw new KeeperException.BadVersionException(path); } - executor.execute(() -> { + log.debug("[{}] Updating -- current version: {}", path, currentVersion); + mockZNode.updateData(data); + Stat stat = mockZNode.getStat(); + toNotify.addAll(getWatchers(path)); + watchers.removeAll(path); + + runNotifications(() -> { triggerPersistentWatches(path, null, EventType.NodeDataChanged); toNotify.forEach(watcher -> watcher .process(new WatchedEvent(EventType.NodeDataChanged, KeeperState.SyncConnected, path))); }); - return createStatForZNode(newZNode); + return stat; } @Override public void setData(final String path, final byte[] data, int version, final StatCallback cb, final Object ctx) { - if (stopped) { + if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } - - executor.execute(() -> { + runInExecutorAsync(() -> { try { final Set toNotify = Sets.newHashSet(); Stat stat; - lock(); - try { - Optional failure = programmedFailure(Op.SET, path); - if (failure.isPresent()) { - unlockIfLocked(); - cb.processResult(failure.get().intValue(), path, ctx, null); - return; - } else if (stopped) { - unlockIfLocked(); - cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); - return; - } - - if (!tree.containsKey(path)) { - unlockIfLocked(); - cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); - return; - } + Optional failure = programmedFailure(Op.SET, path); + if (failure.isPresent()) { + cb.processResult(failure.get().intValue(), path, ctx, null); + return; + } else if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); + return; + } - MockZNode mockZNode = tree.get(path); - int currentVersion = mockZNode.getVersion(); + if (!tree.containsKey(path)) { + cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); + return; + } - // Check version - if (version != -1 && version != currentVersion) { - log.debug("[{}] Current version: {} -- Expected: {}", path, currentVersion, version); - unlockIfLocked(); - cb.processResult(KeeperException.Code.BADVERSION.intValue(), path, ctx, null); - return; - } + MockZNode mockZNode = tree.get(path); + int currentVersion = mockZNode.getVersion(); - log.debug("[{}] Updating -- current version: {}", path, currentVersion); - MockZNode newZNode = MockZNode.of(data, currentVersion + 1, mockZNode.getEphemeralOwner()); - tree.put(path, newZNode); - stat = createStatForZNode(newZNode); - } finally { - unlockIfLocked(); + // Check version + if (version != -1 && version != currentVersion) { + log.debug("[{}] Current version: {} -- Expected: {}", path, currentVersion, version); + Stat currentStat = mockZNode.getStat(); + cb.processResult(KeeperException.Code.BADVERSION.intValue(), path, ctx, currentStat); + return; } + + log.debug("[{}] Updating -- current version: {}", path, currentVersion); + mockZNode.updateData(data); + stat = mockZNode.getStat(); cb.processResult(0, path, ctx, stat); - toNotify.addAll(watchers.get(path)); + toNotify.addAll(getWatchers(path)); watchers.removeAll(path); - for (Watcher watcher : toNotify) { - watcher.process(new WatchedEvent(EventType.NodeDataChanged, KeeperState.SyncConnected, path)); - } + runNotifications(() -> { + triggerPersistentWatches(path, null, EventType.NodeDataChanged); - triggerPersistentWatches(path, null, EventType.NodeDataChanged); + for (Watcher watcher : toNotify) { + watcher.process(new WatchedEvent(EventType.NodeDataChanged, KeeperState.SyncConnected, path)); + } + }); } catch (Throwable ex) { log.error("Update data : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); @@ -878,50 +842,49 @@ public void setData(final String path, final byte[] data, int version, final Sta @Override public void delete(final String path, int version) throws InterruptedException, KeeperException { + runInExecutorReturningValue(() -> { + internalDelete(path, version); + return null; + }); + } + + private void internalDelete(String path, int version) throws KeeperException { maybeThrowProgrammedFailure(Op.DELETE, path); final Set toNotifyDelete; final Set toNotifyParent; final String parent; - lock(); - try { - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } else if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(path); - } else if (hasChildren(path)) { - throw new KeeperException.NotEmptyException(path); - } + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); + } else if (!tree.containsKey(path)) { + throw new KeeperException.NoNodeException(path); + } else if (hasChildren(path)) { + throw new KeeperException.NotEmptyException(path); + } - if (version != -1) { - int currentVersion = tree.get(path).getVersion(); - if (version != currentVersion) { - throw new KeeperException.BadVersionException(path); - } + if (version != -1) { + int currentVersion = tree.get(path).getVersion(); + if (version != currentVersion) { + throw new KeeperException.BadVersionException(path); } + } - tree.remove(path); - - toNotifyDelete = Sets.newHashSet(); - toNotifyDelete.addAll(watchers.get(path)); + parent = getParentName(path); + tree.remove(path); + tree.get(parent).getChildren().remove(getNodeName(path)); - toNotifyParent = Sets.newHashSet(); - parent = path.substring(0, path.lastIndexOf("/")); - if (!parent.isEmpty()) { - toNotifyParent.addAll(watchers.get(parent)); - } + toNotifyDelete = Sets.newHashSet(); + toNotifyDelete.addAll(getWatchers(path)); - watchers.removeAll(path); - } finally { - unlockIfLocked(); + toNotifyParent = Sets.newHashSet(); + if (!ROOT_PATH.equals(parent)) { + toNotifyParent.addAll(getWatchers(parent)); } - executor.execute(() -> { - if (stopped) { - return; - } + watchers.removeAll(path); + runNotifications(() -> { for (Watcher watcher1 : toNotifyDelete) { watcher1.process(new WatchedEvent(EventType.NodeDeleted, KeeperState.SyncConnected, path)); } @@ -935,179 +898,209 @@ public void delete(final String path, int version) throws InterruptedException, @Override public void delete(final String path, int version, final VoidCallback cb, final Object ctx) { - Runnable r = () -> { + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); + return; + } + runInExecutorAsync(() -> { try { - lock(); final Set toNotifyDelete = Sets.newHashSet(); - toNotifyDelete.addAll(watchers.get(path)); + toNotifyDelete.addAll(getWatchers(path)); final Set toNotifyParent = Sets.newHashSet(); - final String parent = path.substring(0, path.lastIndexOf("/")); - if (!parent.isEmpty()) { - toNotifyParent.addAll(watchers.get(parent)); + final String parent = getParentName(path); + if (!ROOT_PATH.equals(parent)) { + toNotifyParent.addAll(getWatchers(parent)); } watchers.removeAll(path); Optional failure = programmedFailure(Op.DELETE, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx); - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); } else if (!tree.containsKey(path)) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx); } else if (hasChildren(path)) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NOTEMPTY.intValue(), path, ctx); } else { if (version != -1) { int currentVersion = tree.get(path).getVersion(); if (version != currentVersion) { - unlockIfLocked(); cb.processResult(KeeperException.Code.BADVERSION.intValue(), path, ctx); return; } } tree.remove(path); - - unlockIfLocked(); + tree.get(parent).getChildren().remove(getNodeName(path)); cb.processResult(0, path, ctx); - toNotifyDelete.forEach(watcher -> watcher - .process(new WatchedEvent(EventType.NodeDeleted, KeeperState.SyncConnected, path))); - toNotifyParent.forEach(watcher -> watcher - .process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, - parent))); - triggerPersistentWatches(path, parent, EventType.NodeDeleted); + runNotifications(() -> { + triggerPersistentWatches(path, parent, EventType.NodeDeleted); + toNotifyDelete.forEach(watcher -> watcher + .process(new WatchedEvent(EventType.NodeDeleted, KeeperState.SyncConnected, path))); + toNotifyParent.forEach(watcher -> watcher + .process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, + parent))); + }); } } catch (Throwable ex) { log.error("delete path : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx); - } finally { - unlockIfLocked(); } - }; - - try { - executor.execute(r); - } catch (RejectedExecutionException ree) { - cb.processResult(KeeperException.Code.SESSIONEXPIRED.intValue(), path, ctx); - } - + }); } @Override public void multi(Iterable ops, AsyncCallback.MultiCallback cb, Object ctx) { - try { - List res = multi(ops); - cb.processResult(KeeperException.Code.OK.intValue(), null, ctx, res); - } catch (Exception e) { - cb.processResult(KeeperException.Code.APIERROR.intValue(), null, ctx, null); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), null, ctx, null); + return; } + runInExecutorAsync(() -> { + try { + List res = multi(ops); + cb.processResult(KeeperException.Code.OK.intValue(), null, ctx, res); + } catch (Exception e) { + cb.processResult(KeeperException.Code.APIERROR.intValue(), null, ctx, null); + } + }); } @Override public List multi(Iterable ops) throws InterruptedException, KeeperException { + return runInExecutorReturningValue(() -> internalMulti(ops)); + } + + private List internalMulti(Iterable ops) { List res = new ArrayList<>(); - try { - for (org.apache.zookeeper.Op op : ops) { - switch (op.getType()) { - case ZooDefs.OpCode.create -> { + for (org.apache.zookeeper.Op op : ops) { + switch (op.getType()) { + case ZooDefs.OpCode.create -> { + handleOperation("create", op, () -> { org.apache.zookeeper.Op.Create opc = ((org.apache.zookeeper.Op.Create) op); CreateMode cm = CreateMode.fromFlag(opc.flags); - String path = this.create(op.getPath(), opc.data, null, cm); + String path = create(op.getPath(), opc.data, null, cm); res.add(new OpResult.CreateResult(path)); - } - case ZooDefs.OpCode.delete -> { - this.delete(op.getPath(), (int) FieldUtils.readField(op, "version", true)); + }, res); + } + case ZooDefs.OpCode.delete -> { + handleOperation("delete", op, () -> { + DeleteRequest deleteRequest = (DeleteRequest) op.toRequestRecord(); + delete(op.getPath(), deleteRequest.getVersion()); res.add(new OpResult.DeleteResult()); - } - case ZooDefs.OpCode.setData -> { - Stat stat = this.setData( - op.getPath(), - (byte[]) FieldUtils.readField(op, "data", true), - (int) FieldUtils.readField(op, "version", true)); + }, res); + } + case ZooDefs.OpCode.setData -> { + handleOperation("setData", op, () -> { + SetDataRequest setDataRequest = (SetDataRequest) op.toRequestRecord(); + Stat stat = setData(op.getPath(), setDataRequest.getData(), setDataRequest.getVersion()); res.add(new OpResult.SetDataResult(stat)); - } - case ZooDefs.OpCode.getChildren -> { - try { - List children = this.getChildren(op.getPath(), null); - res.add(new OpResult.GetChildrenResult(children)); - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - } - } - case ZooDefs.OpCode.getData -> { - Stat stat = new Stat(); - try { - byte[] payload = this.getData(op.getPath(), null, stat); - res.add(new OpResult.GetDataResult(payload, stat)); - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - } - } + }, res); + } + case ZooDefs.OpCode.getChildren -> { + handleOperation("getChildren", op, () -> { + List children = getChildren(op.getPath(), null); + res.add(new OpResult.GetChildrenResult(children)); + }, res); + } + case ZooDefs.OpCode.getData -> { + Stat stat = new Stat(); + handleOperation("getData", op, () -> { + byte[] payload = getData(op.getPath(), null, stat); + res.add(new OpResult.GetDataResult(payload, stat)); + }, res); + } + default -> { + log.error("Unsupported operation for path {} type {} kind {} request {}", op.getPath(), + op.getType(), op.getKind(), op.toRequestRecord()); + res.add(new OpResult.ErrorResult(KeeperException.Code.APIERROR.intValue())); } } - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - int total = Iterables.size(ops); - for (int i = res.size(); i < total; i++) { + } + return res; + } + + interface ZkOpHandler { + void handle() throws KeeperException, InterruptedException; + } + + private void handleOperation(String opName, org.apache.zookeeper.Op op, ZkOpHandler handler, List res) { + try { + handler.handle(); + } catch (Exception e) { + if (e instanceof KeeperException keeperException) { + res.add(new OpResult.ErrorResult(keeperException.code().intValue())); + } else { + log.error("Error handling {} operation for path {} type {} kind {} request {}", opName, op.getPath(), + op.getType(), op.getKind(), op.toRequestRecord(), e); res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); } - } catch (IllegalAccessException e) { - throw new IllegalStateException(e); } - return res; } @Override - public synchronized void addWatch(String basePath, Watcher watcher, AddWatchMode mode) { - persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode)); + public void addWatch(String basePath, Watcher watcher, AddWatchMode mode) { + runInExecutorSync(() -> { + persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode, getSessionId())); + }); } @Override public void addWatch(String basePath, Watcher watcher, AddWatchMode mode, VoidCallback cb, Object ctx) { - if (stopped) { + if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), basePath, ctx); return; } - - executor.execute(() -> { - synchronized (MockZooKeeper.this) { - persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode)); - } - + runInExecutorAsync(() -> { + addWatch(basePath, watcher, mode); cb.processResult(KeeperException.Code.OK.intValue(), basePath, ctx); }); } + public synchronized void increaseRefCount() { + referenceCount++; + } + + public synchronized MockZooKeeper registerCloseable(AutoCloseable closeable) { + closeables.add(closeable); + return this; + } + @Override - public void close() throws InterruptedException { - shutdown(); + public synchronized void close() throws InterruptedException { + if (--referenceCount <= 0) { + shutdown(); + closeables.forEach(c -> { + try { + c.close(); + } catch (Exception e) { + log.error("Error closing closeable", e); + } + }); + closeables.clear(); + } } public void shutdown() throws InterruptedException { - lock(); - try { - stopped = true; - tree.clear(); - watchers.clear(); + if (stopped.compareAndSet(false, true)) { + Future shutdownTask = executor.submit(() -> { + tree.clear(); + watchers.clear(); + persistentWatchers.clear(); + }); try { - executor.shutdownNow(); - executor.awaitTermination(5, TimeUnit.SECONDS); - } catch (InterruptedException ex) { - log.error("MockZooKeeper shutdown had error", ex); + shutdownTask.get(); + } catch (ExecutionException e) { + log.error("Error shutting down", e); } - } finally { - unlockIfLocked(); + MoreExecutors.shutdownAndAwaitTermination(executor, 10, TimeUnit.SECONDS); } } Optional programmedFailure(Op op, String path) { - KeeperException.Code error = this.alwaysFail.get(); + KeeperException.Code error = alwaysFail.get(); if (error != KeeperException.Code.OK) { return Optional.of(error); } @@ -1144,26 +1137,17 @@ public void delay(long millis, BiPredicate predicate) { } public void setAlwaysFail(KeeperException.Code rc) { - this.alwaysFail.set(rc); + alwaysFail.set(rc); } public void unsetAlwaysFail() { - this.alwaysFail.set(KeeperException.Code.OK); + alwaysFail.set(KeeperException.Code.OK); } public void setSessionId(long id) { sessionId = id; } - @Override - public long getSessionId() { - return sessionId; - } - - private boolean hasChildren(String path) { - return !tree.subMap(path + '/', path + '0').isEmpty(); - } - @Override public String toString() { return "MockZookeeper"; @@ -1182,11 +1166,11 @@ private void checkReadOpDelay() { private void triggerPersistentWatches(String path, String parent, EventType eventType) { persistentWatchers.forEach(w -> { if (w.mode == AddWatchMode.PERSISTENT_RECURSIVE) { - if (path.startsWith(w.getPath())) { + if (path.startsWith(w.path())) { w.watcher.process(new WatchedEvent(eventType, KeeperState.SyncConnected, path)); } } else if (w.mode == AddWatchMode.PERSISTENT) { - if (w.getPath().equals(path)) { + if (w.path().equals(path)) { w.watcher.process(new WatchedEvent(eventType, KeeperState.SyncConnected, path)); } @@ -1199,5 +1183,26 @@ private void triggerPersistentWatches(String path, String parent, EventType even }); } + public void deleteEphemeralNodes(long sessionId) { + if (sessionId != NOT_EPHEMERAL) { + runInExecutorSync(() -> { + tree.values().removeIf(zNode -> zNode.getEphemeralOwner() == sessionId); + }); + } + } + + + public void deleteWatchers(long sessionId) { + runInExecutorSync(() -> { + // remove all persistent watchers for the session + persistentWatchers.removeIf(w -> w.sessionId == sessionId); + // remove all watchers for the session + List> watchersForSession = + watchers.entries().stream().filter(e -> e.getValue().sessionId == sessionId).toList(); + watchersForSession + .forEach(e -> watchers.remove(e.getKey(), e.getValue())); + }); + } + private static final Logger log = LoggerFactory.getLogger(MockZooKeeper.class); } diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java index a286a75aa9103..c812423b7280d 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java @@ -40,7 +40,7 @@ public class MockZooKeeperSession extends ZooKeeper { private MockZooKeeper mockZooKeeper; - private long sessionId = 0L; + private long sessionId = 1L; private static final Objenesis objenesis = new ObjenesisStd(); @@ -59,6 +59,9 @@ public static MockZooKeeperSession newInstance(MockZooKeeper mockZooKeeper, bool mockZooKeeperSession.mockZooKeeper = mockZooKeeper; mockZooKeeperSession.sessionId = sessionIdGenerator.getAndIncrement(); mockZooKeeperSession.closeMockZooKeeperOnClose = closeMockZooKeeperOnClose; + if (closeMockZooKeeperOnClose) { + mockZooKeeper.increaseRefCount(); + } return mockZooKeeperSession; } @@ -81,17 +84,22 @@ public States getState() { @Override public void register(Watcher watcher) { - mockZooKeeper.register(watcher); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.register(watcher); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public String create(String path, byte[] data, List acl, CreateMode createMode) throws KeeperException, InterruptedException { try { - mockZooKeeper.overrideEpheralOwner(getSessionId()); + mockZooKeeper.overrideSessionId(getSessionId()); return mockZooKeeper.create(path, data, acl, createMode); } finally { - mockZooKeeper.removeEpheralOwnerOverride(); + mockZooKeeper.removeSessionIdOverride(); } } @@ -99,134 +107,257 @@ public String create(String path, byte[] data, List acl, CreateMode createM public void create(final String path, final byte[] data, final List acl, CreateMode createMode, final AsyncCallback.StringCallback cb, final Object ctx) { try { - mockZooKeeper.overrideEpheralOwner(getSessionId()); + mockZooKeeper.overrideSessionId(getSessionId()); mockZooKeeper.create(path, data, acl, createMode, cb, ctx); } finally { - mockZooKeeper.removeEpheralOwnerOverride(); + mockZooKeeper.removeSessionIdOverride(); } } @Override - public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException { - return mockZooKeeper.getData(path, watcher, stat); + public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException, InterruptedException { + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.getData(path, watcher, stat); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void getData(final String path, boolean watch, final DataCallback cb, final Object ctx) { - mockZooKeeper.getData(path, watch, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.getData(path, watch, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void getData(final String path, final Watcher watcher, final DataCallback cb, final Object ctx) { - mockZooKeeper.getData(path, watcher, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.getData(path, watcher, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void getChildren(final String path, final Watcher watcher, final ChildrenCallback cb, final Object ctx) { - mockZooKeeper.getChildren(path, watcher, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.getChildren(path, watcher, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override - public List getChildren(String path, Watcher watcher) throws KeeperException { - return mockZooKeeper.getChildren(path, watcher); + public List getChildren(String path, Watcher watcher) throws KeeperException, InterruptedException { + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.getChildren(path, watcher); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public List getChildren(String path, boolean watch) throws KeeperException, InterruptedException { - return mockZooKeeper.getChildren(path, watch); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.getChildren(path, watch); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void getChildren(final String path, boolean watcher, final Children2Callback cb, final Object ctx) { - mockZooKeeper.getChildren(path, watcher, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.getChildren(path, watcher, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { - return mockZooKeeper.exists(path, watch); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.exists(path, watch); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException { - return mockZooKeeper.exists(path, watcher); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.exists(path, watcher); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void exists(String path, boolean watch, StatCallback cb, Object ctx) { - mockZooKeeper.exists(path, watch, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.exists(path, watch, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { - mockZooKeeper.exists(path, watcher, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.exists(path, watcher, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void sync(String path, VoidCallback cb, Object ctx) { - mockZooKeeper.sync(path, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.sync(path, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public Stat setData(final String path, byte[] data, int version) throws KeeperException, InterruptedException { - return mockZooKeeper.setData(path, data, version); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.setData(path, data, version); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void setData(final String path, final byte[] data, int version, final StatCallback cb, final Object ctx) { - mockZooKeeper.setData(path, data, version, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.setData(path, data, version, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void delete(final String path, int version) throws InterruptedException, KeeperException { - mockZooKeeper.delete(path, version); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.delete(path, version); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void delete(final String path, int version, final VoidCallback cb, final Object ctx) { - mockZooKeeper.delete(path, version, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.delete(path, version, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void multi(Iterable ops, AsyncCallback.MultiCallback cb, Object ctx) { - mockZooKeeper.multi(ops, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.multi(ops, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public List multi(Iterable ops) throws InterruptedException, KeeperException { - return mockZooKeeper.multi(ops); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.multi(ops); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void addWatch(String basePath, Watcher watcher, AddWatchMode mode, VoidCallback cb, Object ctx) { - mockZooKeeper.addWatch(basePath, watcher, mode, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.addWatch(basePath, watcher, mode, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void addWatch(String basePath, Watcher watcher, AddWatchMode mode) throws KeeperException, InterruptedException { - mockZooKeeper.addWatch(basePath, watcher, mode); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.addWatch(basePath, watcher, mode); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void addWatch(String basePath, AddWatchMode mode) throws KeeperException, InterruptedException { - mockZooKeeper.addWatch(basePath, mode); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.addWatch(basePath, mode); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void addWatch(String basePath, AddWatchMode mode, VoidCallback cb, Object ctx) { - mockZooKeeper.addWatch(basePath, mode, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.addWatch(basePath, mode, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void close() throws InterruptedException { - if (closeMockZooKeeperOnClose) { - mockZooKeeper.close(); - } + internalClose(false); } public void shutdown() throws InterruptedException { - if (closeMockZooKeeperOnClose) { - mockZooKeeper.shutdown(); + internalClose(true); + } + + private void internalClose(boolean shutdown) throws InterruptedException { + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.deleteEphemeralNodes(getSessionId()); + mockZooKeeper.deleteWatchers(getSessionId()); + if (closeMockZooKeeperOnClose) { + if (shutdown) { + mockZooKeeper.shutdown(); + } else { + mockZooKeeper.close(); + } + } + } finally { + mockZooKeeper.removeSessionIdOverride(); } } From 368f430fa9e15d2c7cffd048751849d646f15dfd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 23:11:26 -0800 Subject: [PATCH 35/44] [improve][meta] Simplify getting parent path in ZKMetadataStore without using java.io.File (#23996) (cherry picked from commit 4bfdcd85ff5c9f21ba4fd976adf7785465053151) (cherry picked from commit f5d7309fe64752dbb7eefa04295fbcf56879479e) --- .../org/apache/pulsar/metadata/impl/ZKMetadataStore.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 21128c77477b4..22ae0f122cdb8 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -19,7 +19,6 @@ package org.apache.pulsar.metadata.impl; import com.google.common.annotations.VisibleForTesting; -import java.io.File; import java.util.Collections; import java.util.EnumSet; import java.util.List; @@ -608,8 +607,10 @@ private static void asyncCreateFullPathOptimistic(final ZooKeeper zk, final Stri if (rc != Code.NONODE.intValue()) { callback.processResult(rc, path, ctx, name); } else { - String parent = (new File(originalPath)).getParent().replace("\\", "/"); - + String parent = parent(originalPath); + if (parent == null) { + parent = "/"; + } // Create parent nodes as "CONTAINER" so that ZK will automatically delete them when they're empty asyncCreateFullPathOptimistic(zk, parent, new byte[0], CreateMode.CONTAINER, (rc1, path1, ctx1, name1) -> { From 89fea5b930cebb1520a785c80441e8a2d65ae054 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Tue, 18 Feb 2025 17:59:52 +0800 Subject: [PATCH 36/44] [fix][client] fix retry topic with exclusive mode. (#23859) (cherry picked from commit 5a59ab7768e11db6ed92dada78b398feca9e24fc) (cherry picked from commit 8411eef6aaf2b3fcdb172941b139453034575461) --- .../pulsar/client/api/RetryTopicTest.java | 40 +++++++++++++++++++ .../pulsar/client/impl/ConsumerBase.java | 8 ++++ 2 files changed, 48 insertions(+) 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 d0e72deb87fc2..2b897760b6f00 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 @@ -136,6 +136,46 @@ public void testRetryTopic() throws Exception { checkConsumer.close(); } + /** + * Retry topic feature relies on the delay queue feature when consumer produce a delayed message + * to the retry topic. The delay queue feature is only supported in shared and key-shared subscription type. + * As a result, the subscription type of the retry topic should be shared or key-shared. + * @throws Exception + */ + @Test + public void testRetryTopicWithExclusiveMode() throws Exception { + final String topic = "persistent://my-property/my-ns/retry-topic-exclusive"; + final int maxRedeliveryCount = 2; + + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) + .topic(topic) + .subscriptionName("my-subscription") + .subscriptionType(SubscriptionType.Exclusive) + .enableRetry(true) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build()) + .receiverQueueSize(100) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .create(); + + producer.send("Hello Pulsar".getBytes()); + producer.close(); + + // receive message and set delay to 5 seconds + Message message = consumer.receive(); + long timestamp = System.currentTimeMillis(); + consumer.reconsumeLater(message, 4, TimeUnit.SECONDS); + + // receive message and check the delay is at least 4 seconds + consumer.receive(); + long delay = System.currentTimeMillis() - timestamp; + assertTrue(delay >= 2000); + consumer.close(); + } + @Data public static class Foo { @Nullable diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index df8eb28b95508..7cf6f079f717a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -46,6 +46,7 @@ import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; import org.apache.pulsar.client.api.ConsumerEventListener; +import org.apache.pulsar.client.api.DeadLetterPolicy; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageIdAdv; @@ -783,6 +784,13 @@ private boolean isCumulativeAcknowledgementAllowed(SubscriptionType type) { } protected SubType getSubType() { + // For retry topic, we always use Shared subscription + // Because we will produce delayed messages to retry topic. + DeadLetterPolicy deadLetterPolicy = conf.getDeadLetterPolicy(); + if (deadLetterPolicy != null && topic.equals(deadLetterPolicy.getRetryLetterTopic())) { + return SubType.Shared; + } + SubscriptionType type = conf.getSubscriptionType(); switch (type) { case Exclusive: From 6d640ea509218b3f33f26980632406284390762a Mon Sep 17 00:00:00 2001 From: nikhil-ctds <151718832+nikhil-ctds@users.noreply.github.com> Date: Wed, 19 Feb 2025 14:59:24 +0000 Subject: [PATCH 37/44] Removed update-datastax-license-version.sh & skip license check for datastax --- .../server/src/assemble/LICENSE.bin.txt | 25 ---------- pulsar-sql/presto-distribution/LICENSE | 1 - src/check-binary-license.sh | 5 ++ src/update-datastax-license-version.sh | 50 ------------------- 4 files changed, 5 insertions(+), 76 deletions(-) delete mode 100755 src/update-datastax-license-version.sh diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index d73e1c6862312..1297f604ea65f 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -577,31 +577,6 @@ Bouncy Castle License - org.bouncycastle-bcprov-jdk18on-1.78.1.jar - org.bouncycastle-bcutil-jdk18on-1.78.1.jar -Datastax - - com.datastax.oss-managed-ledger-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-common-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-config-validation-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-docs-tools-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-api-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-instance-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-proto-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-runtime-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-secrets-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-utils-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-worker-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-io-core-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-metadata-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-proxy-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-testclient-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-common-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-testmocks-3.1.4.13-SNAPSHOT.jar - - com.datastax.oss-jetcd-core-shaded-3.1.4.13-SNAPSHOT.jar - ------------------------ Additionaly, Netty includes code with the following licenses: diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 2cdf9c5bf2be5..ee8348a054ad6 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -272,7 +272,6 @@ The Apache Software License, Version 2.0 - grpc-stub-1.55.3.jar - grpc-util-1.60.0.jar * JEtcd - shaded - - jetcd-core-shaded-3.1.4.13-SNAPSHOT.jar * Vertx - shaded * Joda Time - joda-time-2.10.10.jar diff --git a/src/check-binary-license.sh b/src/check-binary-license.sh index 3a6d266345f30..c7f3eea90d26a 100755 --- a/src/check-binary-license.sh +++ b/src/check-binary-license.sh @@ -66,6 +66,11 @@ for J in $JARS; do continue fi + echo $J | grep -q "com.datastax.oss" + if [ $? == 0 ]; then + continue + fi + echo "$LICENSE" | grep -q $J if [ $? != 0 ]; then echo $J unaccounted for in LICENSE diff --git a/src/update-datastax-license-version.sh b/src/update-datastax-license-version.sh deleted file mode 100755 index e500016ebda24..0000000000000 --- a/src/update-datastax-license-version.sh +++ /dev/null @@ -1,50 +0,0 @@ -#!/usr/bin/env bash -# -# 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. -# - -set -e -if [ "$#" -ne 3 ]; then - echo "Usage: $0 " - exit 1 -fi - -FILENAME="$1" -CURRENT_VERSION="$2" -NEXT_VERSION="$3" - -echo "$FILENAME" -echo "$CURRENT_VERSION" -echo "$NEXT_VERSION" - -if [ ! -f "$FILENAME" ]; then - echo "Error: File '$FILENAME' not found!" - exit 1 -fi - -while IFS= read -r LINE; do - if [[ "$LINE" == *"$CURRENT_VERSION"* ]]; then - UPDATED_LINE="${LINE/$CURRENT_VERSION/$NEXT_VERSION}" - echo "$UPDATED_LINE" - else - echo "$LINE" - fi -done < "$FILENAME" > "$FILENAME.new" - -mv "$FILENAME.new" "$FILENAME" -echo "License file '$FILENAME' updated successfully." \ No newline at end of file From 273bba58782914b5c0a4b4a6c018931ba49dcc90 Mon Sep 17 00:00:00 2001 From: nikhil-ctds <151718832+nikhil-ctds@users.noreply.github.com> Date: Thu, 20 Feb 2025 05:25:57 +0000 Subject: [PATCH 38/44] Fix cherry-pick issue --- .../pulsar/metadata/BaseMetadataStoreTest.java | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index 8f45e6e473a32..c4bd1959accf8 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -153,18 +153,6 @@ protected Object[][] filterImplementations(String... providers) { .toArray(Object[][]::new); } - @DataProvider(name = "zkImpls") - public Object[][] zkImplementations() { - return filterImplementations("ZooKeeper", "MockZooKeeper"); - } - - protected Object[][] filterImplementations(String... providers) { - Set providersSet = Set.of(providers); - return Arrays.stream(allImplementations()) - .filter(impl -> providersSet.contains(impl[0])) - .toArray(Object[][]::new); - } - private synchronized String getEtcdClusterConnectString() { if (!running) { return null; From 1614f7fd16805374b16d2fcea5fb09fe391dbb34 Mon Sep 17 00:00:00 2001 From: nikhil-ctds <151718832+nikhil-ctds@users.noreply.github.com> Date: Thu, 20 Feb 2025 05:47:47 +0000 Subject: [PATCH 39/44] Fix cherry-pick issue --- .../buffer/impl/TopicTransactionBuffer.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 58a31a4d2ee27..c44c64f3d50b4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -102,6 +102,7 @@ public class TopicTransactionBuffer extends TopicTransactionBufferState implemen private final AbortedTxnProcessor snapshotAbortedTxnProcessor; + private final AbortedTxnProcessor.SnapshotType snapshotType; private final MaxReadPositionCallBack maxReadPositionCallBack; private static AbortedTxnProcessor createSnapshotProcessor(PersistentTopic topic) { @@ -110,13 +111,19 @@ private static AbortedTxnProcessor createSnapshotProcessor(PersistentTopic topic : new SingleSnapshotAbortedTxnProcessorImpl(topic); } + private static AbortedTxnProcessor.SnapshotType determineSnapshotType(PersistentTopic topic) { + return topic.getBrokerService().getPulsar().getConfiguration().isTransactionBufferSegmentedSnapshotEnabled() + ? AbortedTxnProcessor.SnapshotType.Segment + : AbortedTxnProcessor.SnapshotType.Single; + } public TopicTransactionBuffer(PersistentTopic topic) { - this(topic, createSnapshotProcessor(topic)); + this(topic, createSnapshotProcessor(topic), determineSnapshotType(topic)); } @VisibleForTesting - TopicTransactionBuffer(PersistentTopic topic, AbortedTxnProcessor snapshotAbortedTxnProcessor) { + TopicTransactionBuffer(PersistentTopic topic, AbortedTxnProcessor snapshotAbortedTxnProcessor, + AbortedTxnProcessor.SnapshotType snapshotType) { super(State.None); this.topic = topic; this.timer = topic.getBrokerService().getPulsar().getTransactionTimer(); @@ -126,6 +133,7 @@ public TopicTransactionBuffer(PersistentTopic topic) { .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis(); this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); this.snapshotAbortedTxnProcessor = snapshotAbortedTxnProcessor; + this.snapshotType = snapshotType; this.maxReadPositionCallBack = topic.getMaxReadPositionCallBack(); this.recover(); } From ea01af24ce565127a5e16d8a37e47b8df1a53faf Mon Sep 17 00:00:00 2001 From: nikhil-ctds <151718832+nikhil-ctds@users.noreply.github.com> Date: Thu, 20 Feb 2025 06:21:35 +0000 Subject: [PATCH 40/44] Fix cherry-pick issue --- .../broker/service/BrokerServiceTest.java | 60 ------------------- 1 file changed, 60 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 34ddc6d9466c8..694d3608ea5c8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -1664,41 +1664,6 @@ public void testDynamicConfigurationsForceDeleteTenantAllowed() throws Exception }); } - @Test - public void testIsSystemTopicAllowAutoTopicCreationAsync() throws Exception { - BrokerService brokerService = pulsar.getBrokerService(); - assertFalse(brokerService.isAllowAutoTopicCreationAsync( - ServiceUnitStateChannelImpl.TOPIC).get()); - assertTrue(brokerService.isAllowAutoTopicCreationAsync( - "persistent://pulsar/system/my-system-topic").get()); - } - - @Test - public void testDuplicateAcknowledgement() throws Exception { - final String ns = "prop/ns-test"; - - admin.namespaces().createNamespace(ns, 2); - final String topicName = "persistent://prop/ns-test/duplicated-acknowledgement-test"; - @Cleanup - Producer producer = pulsarClient.newProducer() - .topic(topicName) - .create(); - @Cleanup - Consumer consumer1 = pulsarClient.newConsumer() - .topic(topicName) - .subscriptionName("sub-1") - .acknowledgmentGroupTime(0, TimeUnit.SECONDS) - .subscriptionType(SubscriptionType.Shared) - .isAckReceiptEnabled(true) - .subscribe(); - producer.send("1".getBytes(StandardCharsets.UTF_8)); - Message message = consumer1.receive(); - consumer1.acknowledge(message); - consumer1.acknowledge(message); - assertEquals(admin.topics().getStats(topicName).getSubscriptions() - .get("sub-1").getUnackedMessages(), 0); - } - @Test public void testBlockedConsumerOnUnackedMsgs() throws Exception { final String ns = "prop/ns-test"; @@ -1740,31 +1705,6 @@ public void testBlockedConsumerOnUnackedMsgs() throws Exception { assertFalse(subscriptionStats.getConsumers().get(0).isBlockedConsumerOnUnackedMsgs()); } - @Test - public void testUnsubscribeNonDurableSub() throws Exception { - final String ns = "prop/ns-test"; - final String topic = ns + "/testUnsubscribeNonDurableSub"; - - admin.namespaces().createNamespace(ns, 2); - admin.topics().createPartitionedTopic(String.format("persistent://%s", topic), 1); - - pulsarClient.newProducer(Schema.STRING).topic(topic).create().close(); - @Cleanup - Consumer consumer = pulsarClient - .newConsumer(Schema.STRING) - .topic(topic) - .subscriptionMode(SubscriptionMode.NonDurable) - .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) - .subscriptionName("sub1") - .subscriptionType(SubscriptionType.Shared) - .subscribe(); - try { - consumer.unsubscribe(); - } catch (Exception ex) { - fail("Unsubscribe failed"); - } - } - // this test is disabled since it is flaky @Test(enabled = false) public void testMetricsPersistentTopicLoadFails() throws Exception { From 705550363736aa0e0039669e129211e6d2e28d66 Mon Sep 17 00:00:00 2001 From: nikhil-ctds <151718832+nikhil-ctds@users.noreply.github.com> Date: Thu, 20 Feb 2025 07:01:46 +0000 Subject: [PATCH 41/44] Fix cherry-pick issue --- .../transaction/buffer/impl/TransactionPersistentTopicTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionPersistentTopicTest.java index 8538ea2bd9984..508423adce4d8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionPersistentTopicTest.java @@ -83,7 +83,7 @@ public void testNoOrphanClosedTopicIfTxnInternalFailed() { }).when(abortedTxnProcessor).recoverFromSnapshot(); when(abortedTxnProcessor.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); return new TopicTransactionBuffer( - (PersistentTopic) originTopic, abortedTxnProcessor); + (PersistentTopic) originTopic, abortedTxnProcessor, AbortedTxnProcessor.SnapshotType.Single); }; TransactionBufferProvider originalTransactionBufferProvider = pulsar.getTransactionBufferProvider(); pulsar.setTransactionBufferProvider(mockTransactionBufferProvider); From 6f0ae5062d64f00a40fa5001fdc3b3e65e3a0ca0 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 19 Feb 2025 15:18:56 +0200 Subject: [PATCH 42/44] Fix presto-distribution LICENSE (cherry picked from commit ee834f82b9369fea0720ae96748d3757a8b37043) --- pulsar-sql/presto-distribution/LICENSE | 52 +++++++++++++------------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index ee8348a054ad6..22b0a25c200f8 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -231,33 +231,33 @@ The Apache Software License, Version 2.0 - commons-compress-1.26.0.jar - commons-lang3-3.11.jar * Netty - - 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-buffer-4.1.118.Final.jar + - netty-codec-4.1.118.Final.jar + - netty-codec-dns-4.1.118.Final.jar + - netty-codec-http-4.1.118.Final.jar + - netty-codec-haproxy-4.1.118.Final.jar + - netty-codec-socks-4.1.118.Final.jar + - netty-handler-proxy-4.1.118.Final.jar + - netty-common-4.1.118.Final.jar + - netty-handler-4.1.118.Final.jar - netty-reactive-streams-2.0.6.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 - - netty-tcnative-boringssl-static-2.0.69.Final-osx-aarch_64.jar - - 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.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-resolver-4.1.118.Final.jar + - netty-resolver-dns-4.1.118.Final.jar + - netty-resolver-dns-classes-macos-4.1.118.Final.jar + - netty-resolver-dns-native-macos-4.1.118.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.118.Final-osx-x86_64.jar + - netty-tcnative-boringssl-static-2.0.70.Final.jar + - netty-tcnative-boringssl-static-2.0.70.Final-linux-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.70.Final-linux-x86_64.jar + - netty-tcnative-boringssl-static-2.0.70.Final-osx-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.70.Final-osx-x86_64.jar + - netty-tcnative-boringssl-static-2.0.70.Final-windows-x86_64.jar + - netty-tcnative-classes-2.0.70.Final.jar + - netty-transport-4.1.118.Final.jar + - netty-transport-classes-epoll-4.1.118.Final.jar + - netty-transport-native-epoll-4.1.118.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.118.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.118.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 e85cb051ce7e264abb6acca45ee00d4f66c569fa Mon Sep 17 00:00:00 2001 From: Dave Fisher Date: Thu, 6 Jul 2023 13:27:05 -0700 Subject: [PATCH 43/44] Fix cherry-pick issue --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a6fbf06179591..6864bebdba1d9 100644 --- a/pom.xml +++ b/pom.xml @@ -195,7 +195,7 @@ flexible messaging model and an intuitive client API. 0.4.4-hotfix1 3.3.5 2.5.2 - 1.2.4 + 2.16.0 8.12.1 368 2.6.1.Final From 6f601180df2b383ecdacd9f354f5ef8e7852224c Mon Sep 17 00:00:00 2001 From: nikhil-ctds <151718832+nikhil-ctds@users.noreply.github.com> Date: Fri, 21 Feb 2025 15:27:07 +0000 Subject: [PATCH 44/44] Fix licence issue --- src/check-binary-license.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/check-binary-license.sh b/src/check-binary-license.sh index c7f3eea90d26a..bda1271ae4035 100755 --- a/src/check-binary-license.sh +++ b/src/check-binary-license.sh @@ -101,7 +101,7 @@ done if [ "$NO_PRESTO" -ne 1 ]; then # check pulsar sql jars - JARS=$(tar -tf $TARBALL | grep '\.jar' | grep 'trino/' | grep -v pulsar-client | grep -v bouncy-castle-bc | grep -v pulsar-metadata | grep -v 'managed-ledger' | grep -v 'pulsar-client-admin' | grep -v 'pulsar-client-api' | grep -v 'pulsar-functions-api' | grep -v 'pulsar-presto-connector-original' | grep -v 'pulsar-presto-distribution' | grep -v 'pulsar-common' | grep -v 'pulsar-functions-proto' | grep -v 'pulsar-functions-utils' | grep -v 'pulsar-io-core' | grep -v 'pulsar-transaction-common' | grep -v 'pulsar-package-core' | sed 's!.*/!!' | sort) + JARS=$(tar -tf $TARBALL | grep '\.jar' | grep 'trino/' | grep -v pulsar-client | grep -v bouncy-castle-bc | grep -v pulsar-metadata | grep -v 'managed-ledger' | grep -v 'pulsar-client-admin' | grep -v 'pulsar-client-api' | grep -v 'pulsar-functions-api' | grep -v 'pulsar-presto-connector-original' | grep -v 'pulsar-presto-distribution' | grep -v 'pulsar-common' | grep -v 'pulsar-functions-proto' | grep -v 'pulsar-functions-utils' | grep -v 'pulsar-io-core' | grep -v 'pulsar-transaction-common' | grep -v 'pulsar-package-core' | grep -v jetcd-core-shaded | sed 's!.*/!!' | sort) if [ -n "$JARS" ]; then LICENSEPATH=$(tar -tf $TARBALL | awk '/^[^\/]*\/trino\/LICENSE/') LICENSE=$(tar -O -xf $TARBALL "$LICENSEPATH")