From 9012422bcbaac7b38820ce545cd5a3b4f8b586d0 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 23 Sep 2024 10:44:48 +0800 Subject: [PATCH 001/327] [improve][broker] Remove ConcurrentOpenHashMap and ConcurrentOpenHashSet (#23329) --- .../impl/ManagedLedgerFactoryImpl.java | 4 +- .../broker/namespace/NamespaceService.java | 21 +- .../persistent/MessageDeduplication.java | 15 +- .../stats/ClusterReplicationMetrics.java | 7 +- .../persistent/MessageDuplicationTest.java | 5 +- .../pulsar/client/impl/ConsumerBase.java | 6 +- .../pulsar/client/impl/ConsumerImpl.java | 4 +- .../client/impl/PartitionedProducerImpl.java | 7 +- .../pulsar/client/impl/ProducerBase.java | 7 +- .../AcknowledgementsGroupingTrackerTest.java | 4 +- .../impl/UnAckedMessageTrackerTest.java | 7 +- .../collections/ConcurrentOpenHashMap.java | 658 ---------------- .../collections/ConcurrentOpenHashSet.java | 622 ---------------- .../ConcurrentOpenHashMapTest.java | 700 ------------------ .../ConcurrentOpenHashSetTest.java | 503 ------------- .../pulsar/websocket/WebSocketService.java | 36 +- .../pulsar/websocket/stats/ProxyStats.java | 7 +- 17 files changed, 40 insertions(+), 2573 deletions(-) delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java delete mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java delete mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 586beb412d297..34dd3610d4ec9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -104,7 +104,6 @@ import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.Runnables; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Stat; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -1207,8 +1206,7 @@ public void calculateCursorBacklogs(final TopicName topicName, BookKeeper bk = getBookKeeper().get(); final CountDownLatch allCursorsCounter = new CountDownLatch(1); final long errorInReadingCursor = -1; - ConcurrentOpenHashMap ledgerRetryMap = - ConcurrentOpenHashMap.newBuilder().build(); + final var ledgerRetryMap = new ConcurrentHashMap(); final MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = ledgers.lastEntry().getValue(); final Position lastLedgerPosition = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 0b1661fb9540a..b2ee299bb030e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -109,7 +109,6 @@ import org.apache.pulsar.common.stats.MetricsUtil; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; @@ -150,7 +149,7 @@ public class NamespaceService implements AutoCloseable { public static final String HEARTBEAT_NAMESPACE_FMT_V2 = "pulsar/%s"; public static final String SLA_NAMESPACE_FMT = SLA_NAMESPACE_PROPERTY + "/%s/%s"; - private final ConcurrentOpenHashMap namespaceClients; + private final Map namespaceClients = new ConcurrentHashMap<>(); private final List bundleOwnershipListeners; @@ -204,8 +203,6 @@ public NamespaceService(PulsarService pulsar) { this.loadManager = pulsar.getLoadManager(); this.bundleFactory = new NamespaceBundleFactory(pulsar, Hashing.crc32()); this.ownershipCache = new OwnershipCache(pulsar, this); - this.namespaceClients = - ConcurrentOpenHashMap.newBuilder().build(); this.bundleOwnershipListeners = new CopyOnWriteArrayList<>(); this.bundleSplitListeners = new CopyOnWriteArrayList<>(); this.localBrokerDataCache = pulsar.getLocalMetadataStore().getMetadataCache(LocalBrokerData.class); @@ -461,16 +458,10 @@ public boolean registerNamespace(NamespaceName nsname, boolean ensureOwned) thro } } - private final ConcurrentOpenHashMap>> - findingBundlesAuthoritative = - ConcurrentOpenHashMap.>>newBuilder() - .build(); - private final ConcurrentOpenHashMap>> - findingBundlesNotAuthoritative = - ConcurrentOpenHashMap.>>newBuilder() - .build(); + private final Map>> + findingBundlesAuthoritative = new ConcurrentHashMap<>(); + private final Map>> + findingBundlesNotAuthoritative = new ConcurrentHashMap<>(); /** * Main internal method to lookup and setup ownership of service unit to a broker. @@ -485,7 +476,7 @@ private CompletableFuture> findBrokerServiceUrl( LOG.debug("findBrokerServiceUrl: {} - options: {}", bundle, options); } - ConcurrentOpenHashMap>> targetMap; + Map>> targetMap; if (options.isAuthoritative()) { targetMap = findingBundlesAuthoritative; } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index e8d19d2e2eca1..dfb8b9d2edb12 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -42,7 +42,6 @@ import org.apache.pulsar.broker.service.Topic.PublishContext; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.protocol.Commands; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -101,20 +100,12 @@ public MessageDupUnknownException() { // Map that contains the highest sequenceId that have been sent by each producers. The map will be updated before // the messages are persisted @VisibleForTesting - final ConcurrentOpenHashMap highestSequencedPushed = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); + final Map highestSequencedPushed = new ConcurrentHashMap<>(); // Map that contains the highest sequenceId that have been persistent by each producers. The map will be updated // after the messages are persisted @VisibleForTesting - final ConcurrentOpenHashMap highestSequencedPersisted = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); + final Map highestSequencedPersisted = new ConcurrentHashMap<>(); // Number of persisted entries after which to store a snapshot of the sequence ids map private final int snapshotInterval; @@ -434,7 +425,7 @@ public void resetHighestSequenceIdPushed() { } highestSequencedPushed.clear(); - for (String producer : highestSequencedPersisted.keys()) { + for (String producer : highestSequencedPersisted.keySet()) { highestSequencedPushed.put(producer, highestSequencedPersisted.get(producer)); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/ClusterReplicationMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/ClusterReplicationMetrics.java index 6b274b26b57fb..828cb48be429d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/ClusterReplicationMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/ClusterReplicationMetrics.java @@ -20,23 +20,22 @@ import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; /** */ public class ClusterReplicationMetrics { private final List metricsList; private final String localCluster; - private final ConcurrentOpenHashMap metricsMap; + private final Map metricsMap = new ConcurrentHashMap<>(); public static final String SEPARATOR = "_"; public final boolean metricsEnabled; public ClusterReplicationMetrics(String localCluster, boolean metricsEnabled) { metricsList = new ArrayList<>(); this.localCluster = localCluster; - metricsMap = ConcurrentOpenHashMap.newBuilder() - .build(); this.metricsEnabled = metricsEnabled; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java index e7dcbc602134c..5b1c78574b462 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java @@ -60,7 +60,6 @@ import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.broker.qos.AsyncTokenBucket; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactionServiceFactory; import org.awaitility.Awaitility; import org.testng.Assert; @@ -230,9 +229,7 @@ public void testInactiveProducerRemove() throws Exception { messageDeduplication.purgeInactiveProducers(); assertFalse(inactiveProducers.containsKey(producerName2)); assertFalse(inactiveProducers.containsKey(producerName3)); - field = MessageDeduplication.class.getDeclaredField("highestSequencedPushed"); - field.setAccessible(true); - ConcurrentOpenHashMap highestSequencedPushed = (ConcurrentOpenHashMap) field.get(messageDeduplication); + final var highestSequencedPushed = messageDeduplication.highestSequencedPushed; assertEquals((long) highestSequencedPushed.get(producerName1), 2L); assertFalse(highestSequencedPushed.containsKey(producerName2)); 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 03256a3e139b6..111cbdb8a8ef3 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 @@ -31,6 +31,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -67,7 +68,6 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,7 +88,7 @@ public abstract class ConsumerBase extends HandlerState implements Consumer> incomingMessages; - protected ConcurrentOpenHashMap unAckedChunkedMessageIdSequenceMap; + protected Map unAckedChunkedMessageIdSequenceMap = new ConcurrentHashMap<>(); protected final ConcurrentLinkedQueue>> pendingReceives; protected final int maxReceiverQueueSize; private volatile int currentReceiverQueueSize; @@ -138,8 +138,6 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat this.consumerEventListener = conf.getConsumerEventListener(); // Always use growable queue since items can exceed the advertised size this.incomingMessages = new GrowableArrayBlockingQueue<>(); - this.unAckedChunkedMessageIdSequenceMap = - ConcurrentOpenHashMap.newBuilder().build(); this.executorProvider = executorProvider; this.messageListenerExecutor = conf.getMessageListenerExecutor() == null ? (conf.getSubscriptionType() == SubscriptionType.Key_Shared 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 60b9d145c4897..03ccbae01c276 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 @@ -130,7 +130,6 @@ import org.apache.pulsar.common.util.SafeCollectionUtils; import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -207,8 +206,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle protected volatile boolean paused; - protected ConcurrentOpenHashMap chunkedMessagesMap = - ConcurrentOpenHashMap.newBuilder().build(); + protected Map chunkedMessagesMap = new ConcurrentHashMap<>(); private int pendingChunkedMessageCount = 0; protected long expireTimeOfIncompleteChunkedMessageMillis = 0; private final AtomicBoolean expireChunkMessageTaskScheduled = new AtomicBoolean(false); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java index bf7f1066173f6..2dc826d9e3af3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java @@ -27,9 +27,11 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -52,7 +54,6 @@ import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +61,7 @@ public class PartitionedProducerImpl extends ProducerBase { private static final Logger log = LoggerFactory.getLogger(PartitionedProducerImpl.class); - private final ConcurrentOpenHashMap> producers; + private final Map> producers = new ConcurrentHashMap<>(); private final MessageRouter routerPolicy; private final PartitionedTopicProducerStatsRecorderImpl stats; private TopicMetadata topicMetadata; @@ -76,8 +77,6 @@ public PartitionedProducerImpl(PulsarClientImpl client, String topic, ProducerCo int numPartitions, CompletableFuture> producerCreatedFuture, Schema schema, ProducerInterceptors interceptors) { super(client, topic, conf, producerCreatedFuture, schema, interceptors); - this.producers = - ConcurrentOpenHashMap.>newBuilder().build(); this.topicMetadata = new TopicMetadataImpl(numPartitions); this.routerPolicy = getMessageRouter(); stats = client.getConfiguration().getStatsIntervalSeconds() > 0 diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java index 7dc5f78398434..12e380fdd510c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java @@ -19,7 +19,9 @@ package org.apache.pulsar.client.impl; import static com.google.common.base.Preconditions.checkArgument; +import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -32,7 +34,6 @@ import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.common.protocol.schema.SchemaHash; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; public abstract class ProducerBase extends HandlerState implements Producer { @@ -40,7 +41,7 @@ public abstract class ProducerBase extends HandlerState implements Producer schema; protected final ProducerInterceptors interceptors; - protected final ConcurrentOpenHashMap schemaCache; + protected final Map schemaCache = new ConcurrentHashMap<>(); protected volatile MultiSchemaMode multiSchemaMode = MultiSchemaMode.Auto; protected ProducerBase(PulsarClientImpl client, String topic, ProducerConfigurationData conf, @@ -50,8 +51,6 @@ protected ProducerBase(PulsarClientImpl client, String topic, ProducerConfigurat this.conf = conf; this.schema = schema; this.interceptors = interceptors; - this.schemaCache = - ConcurrentOpenHashMap.newBuilder().build(); if (!conf.isMultiSchema()) { multiSchemaMode = MultiSchemaMode.Disabled; } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java index 514e3dde14070..a62d9e7479852 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java @@ -45,7 +45,6 @@ import org.apache.pulsar.client.util.TimedCompletableFuture; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -62,8 +61,7 @@ public class AcknowledgementsGroupingTrackerTest { public void setup() throws NoSuchFieldException, IllegalAccessException { eventLoopGroup = new NioEventLoopGroup(1); consumer = mock(ConsumerImpl.class); - consumer.unAckedChunkedMessageIdSequenceMap = - ConcurrentOpenHashMap.newBuilder().build(); + consumer.unAckedChunkedMessageIdSequenceMap = new ConcurrentHashMap<>(); cnx = spy(new ClientCnxTest(new ClientConfigurationData(), eventLoopGroup)); PulsarClientImpl client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java index b01fbcb879f80..eaac165818a56 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java @@ -31,13 +31,11 @@ import io.netty.util.concurrent.DefaultThreadFactory; import java.time.Duration; import java.util.HashSet; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; - import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.metrics.InstrumentProvider; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.annotations.Test; @@ -113,8 +111,7 @@ public void testTrackChunkedMessageId() { ChunkMessageIdImpl chunkedMessageId = new ChunkMessageIdImpl(chunkMsgIds[0], chunkMsgIds[chunkMsgIds.length - 1]); - consumer.unAckedChunkedMessageIdSequenceMap = - ConcurrentOpenHashMap.newBuilder().build(); + consumer.unAckedChunkedMessageIdSequenceMap = new ConcurrentHashMap<>(); consumer.unAckedChunkedMessageIdSequenceMap.put(chunkedMessageId, chunkMsgIds); // Redeliver chunked message diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java deleted file mode 100644 index 7f0dbb4379265..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java +++ /dev/null @@ -1,658 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util.collections; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Objects.requireNonNull; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; -import java.util.concurrent.locks.StampedLock; -import java.util.function.BiConsumer; -import java.util.function.Function; - -/** - * Concurrent hash map. - * - *

Provides similar methods as a {@code ConcurrentMap} but since it's an open hash map with linear probing, - * no node allocations are required to store the values. - * - *
- * WARN: method forEach do not guarantee thread safety, nor do the keys and values method. - *
- * The forEach method is specifically designed for single-threaded usage. When iterating over a map - * with concurrent writes, it becomes possible for new values to be either observed or not observed. - * There is no guarantee that if we write value1 and value2, and are able to see value2, then we will also see value1. - * In some cases, it is even possible to encounter two mappings with the same key, - * leading the keys method to return a List containing two identical keys. - * - *
- * It is crucial to understand that the results obtained from aggregate status methods such as keys and values - * are typically reliable only when the map is not undergoing concurrent updates from other threads. - * When concurrent updates are involved, the results of these methods reflect transient states - * that may be suitable for monitoring or estimation purposes, but not for program control. - * @param - */ -@SuppressWarnings("unchecked") -public class ConcurrentOpenHashMap { - - private static final Object EmptyKey = null; - private static final Object DeletedKey = new Object(); - private static final ConcurrentOpenHashMap EmptyMap = new ConcurrentOpenHashMap<>(1, 1); - - /** - * This object is used to delete empty value in this map. - * EmptyValue.equals(null) = true. - */ - private static final Object EmptyValue = new Object() { - - @SuppressFBWarnings - @Override - public boolean equals(Object obj) { - return obj == null; - } - - /** - * This is just for avoiding spotbugs errors - */ - @Override - public int hashCode() { - return super.hashCode(); - } - }; - - private static final int DefaultExpectedItems = 256; - private static final int DefaultConcurrencyLevel = 16; - - private static final float DefaultMapFillFactor = 0.66f; - private static final float DefaultMapIdleFactor = 0.15f; - - private static final float DefaultExpandFactor = 2; - private static final float DefaultShrinkFactor = 2; - - private static final boolean DefaultAutoShrink = false; - - private final Section[] sections; - - public static Builder newBuilder() { - return new Builder<>(); - } - - /** - * Builder of ConcurrentOpenHashMap. - */ - public static class Builder { - int expectedItems = DefaultExpectedItems; - int concurrencyLevel = DefaultConcurrencyLevel; - float mapFillFactor = DefaultMapFillFactor; - float mapIdleFactor = DefaultMapIdleFactor; - float expandFactor = DefaultExpandFactor; - float shrinkFactor = DefaultShrinkFactor; - boolean autoShrink = DefaultAutoShrink; - - public Builder expectedItems(int expectedItems) { - this.expectedItems = expectedItems; - return this; - } - - public Builder concurrencyLevel(int concurrencyLevel) { - this.concurrencyLevel = concurrencyLevel; - return this; - } - - public Builder mapFillFactor(float mapFillFactor) { - this.mapFillFactor = mapFillFactor; - return this; - } - - public Builder mapIdleFactor(float mapIdleFactor) { - this.mapIdleFactor = mapIdleFactor; - return this; - } - - public Builder expandFactor(float expandFactor) { - this.expandFactor = expandFactor; - return this; - } - - public Builder shrinkFactor(float shrinkFactor) { - this.shrinkFactor = shrinkFactor; - return this; - } - - public Builder autoShrink(boolean autoShrink) { - this.autoShrink = autoShrink; - return this; - } - - public ConcurrentOpenHashMap build() { - return new ConcurrentOpenHashMap<>(expectedItems, concurrencyLevel, - mapFillFactor, mapIdleFactor, autoShrink, expandFactor, shrinkFactor); - } - } - - @Deprecated - public ConcurrentOpenHashMap() { - this(DefaultExpectedItems); - } - - @Deprecated - public ConcurrentOpenHashMap(int expectedItems) { - this(expectedItems, DefaultConcurrencyLevel); - } - - @Deprecated - public ConcurrentOpenHashMap(int expectedItems, int concurrencyLevel) { - this(expectedItems, concurrencyLevel, DefaultMapFillFactor, DefaultMapIdleFactor, - DefaultAutoShrink, DefaultExpandFactor, DefaultShrinkFactor); - } - - public ConcurrentOpenHashMap(int expectedItems, int concurrencyLevel, - float mapFillFactor, float mapIdleFactor, - boolean autoShrink, float expandFactor, float shrinkFactor) { - checkArgument(expectedItems > 0); - checkArgument(concurrencyLevel > 0); - checkArgument(expectedItems >= concurrencyLevel); - checkArgument(mapFillFactor > 0 && mapFillFactor < 1); - checkArgument(mapIdleFactor > 0 && mapIdleFactor < 1); - checkArgument(mapFillFactor > mapIdleFactor); - checkArgument(expandFactor > 1); - checkArgument(shrinkFactor > 1); - - int numSections = concurrencyLevel; - int perSectionExpectedItems = expectedItems / numSections; - int perSectionCapacity = (int) (perSectionExpectedItems / mapFillFactor); - this.sections = (Section[]) new Section[numSections]; - - for (int i = 0; i < numSections; i++) { - sections[i] = new Section<>(perSectionCapacity, mapFillFactor, mapIdleFactor, - autoShrink, expandFactor, shrinkFactor); - } - } - - public static ConcurrentOpenHashMap emptyMap() { - return (ConcurrentOpenHashMap) EmptyMap; - } - - long getUsedBucketCount() { - long usedBucketCount = 0; - for (Section s : sections) { - usedBucketCount += s.usedBuckets; - } - return usedBucketCount; - } - - public long size() { - long size = 0; - for (Section s : sections) { - size += s.size; - } - return size; - } - - public long capacity() { - long capacity = 0; - for (Section s : sections) { - capacity += s.capacity; - } - return capacity; - } - - public boolean isEmpty() { - for (Section s : sections) { - if (s.size != 0) { - return false; - } - } - - return true; - } - - public V get(K key) { - requireNonNull(key); - long h = hash(key); - return getSection(h).get(key, (int) h); - } - - public boolean containsKey(K key) { - return get(key) != null; - } - - public V put(K key, V value) { - requireNonNull(key); - requireNonNull(value); - long h = hash(key); - return getSection(h).put(key, value, (int) h, false, null); - } - - public V putIfAbsent(K key, V value) { - requireNonNull(key); - requireNonNull(value); - long h = hash(key); - return getSection(h).put(key, value, (int) h, true, null); - } - - public V computeIfAbsent(K key, Function provider) { - requireNonNull(key); - requireNonNull(provider); - long h = hash(key); - return getSection(h).put(key, null, (int) h, true, provider); - } - - public V remove(K key) { - requireNonNull(key); - long h = hash(key); - return getSection(h).remove(key, null, (int) h); - } - - public boolean remove(K key, Object value) { - requireNonNull(key); - requireNonNull(value); - long h = hash(key); - return getSection(h).remove(key, value, (int) h) != null; - } - - public void removeNullValue(K key) { - remove(key, EmptyValue); - } - - private Section getSection(long hash) { - // Use 32 msb out of long to get the section - final int sectionIdx = (int) (hash >>> 32) & (sections.length - 1); - return sections[sectionIdx]; - } - - public void clear() { - for (int i = 0; i < sections.length; i++) { - sections[i].clear(); - } - } - - /** - * Iterate over all the entries in the map and apply the processor function to each of them. - *

- * Warning: Do Not Guarantee Thread-Safety. - * @param processor the function to apply to each entry - */ - public void forEach(BiConsumer processor) { - for (int i = 0; i < sections.length; i++) { - sections[i].forEach(processor); - } - } - - /** - * @return a new list of all keys (makes a copy) - */ - public List keys() { - List keys = new ArrayList<>((int) size()); - forEach((key, value) -> keys.add(key)); - return keys; - } - - public List values() { - List values = new ArrayList<>((int) size()); - forEach((key, value) -> values.add(value)); - return values; - } - - // A section is a portion of the hash map that is covered by a single - @SuppressWarnings("serial") - private static final class Section extends StampedLock { - // Each item take up 2 continuous array space. - private static final int ITEM_SIZE = 2; - - // Keys and values are stored interleaved in the table array - private volatile Object[] table; - - private volatile int capacity; - private final int initCapacity; - private static final AtomicIntegerFieldUpdater

SIZE_UPDATER = - AtomicIntegerFieldUpdater.newUpdater(Section.class, "size"); - private volatile int size; - private int usedBuckets; - private int resizeThresholdUp; - private int resizeThresholdBelow; - private final float mapFillFactor; - private final float mapIdleFactor; - private final float expandFactor; - private final float shrinkFactor; - private final boolean autoShrink; - - Section(int capacity, float mapFillFactor, float mapIdleFactor, boolean autoShrink, - float expandFactor, float shrinkFactor) { - this.capacity = alignToPowerOfTwo(capacity); - this.initCapacity = this.capacity; - this.table = new Object[ITEM_SIZE * this.capacity]; - this.size = 0; - this.usedBuckets = 0; - this.autoShrink = autoShrink; - this.mapFillFactor = mapFillFactor; - this.mapIdleFactor = mapIdleFactor; - this.expandFactor = expandFactor; - this.shrinkFactor = shrinkFactor; - this.resizeThresholdUp = (int) (this.capacity * mapFillFactor); - this.resizeThresholdBelow = (int) (this.capacity * mapIdleFactor); - } - - V get(K key, int keyHash) { - long stamp = tryOptimisticRead(); - boolean acquiredLock = false; - - // add local variable here, so OutOfBound won't happen - Object[] table = this.table; - // calculate table.length / 2 as capacity to avoid rehash changing capacity - int bucket = signSafeMod(keyHash, table.length / ITEM_SIZE); - - try { - while (true) { - // First try optimistic locking - K storedKey = (K) table[bucket]; - V storedValue = (V) table[bucket + 1]; - - if (!acquiredLock && validate(stamp)) { - // The values we have read are consistent - if (key.equals(storedKey)) { - return storedValue; - } else if (storedKey == EmptyKey) { - // Not found - return null; - } - } else { - // Fallback to acquiring read lock - if (!acquiredLock) { - stamp = readLock(); - acquiredLock = true; - - // update local variable - table = this.table; - bucket = signSafeMod(keyHash, table.length / ITEM_SIZE); - storedKey = (K) table[bucket]; - storedValue = (V) table[bucket + 1]; - } - - if (key.equals(storedKey)) { - return storedValue; - } else if (storedKey == EmptyKey) { - // Not found - return null; - } - } - - bucket = (bucket + ITEM_SIZE) & (table.length - 1); - } - } finally { - if (acquiredLock) { - unlockRead(stamp); - } - } - } - - V put(K key, V value, int keyHash, boolean onlyIfAbsent, Function valueProvider) { - long stamp = writeLock(); - int bucket = signSafeMod(keyHash, capacity); - - // Remember where we find the first available spot - int firstDeletedKey = -1; - - try { - while (true) { - K storedKey = (K) table[bucket]; - V storedValue = (V) table[bucket + 1]; - - if (key.equals(storedKey)) { - if (!onlyIfAbsent) { - // Over written an old value for same key - table[bucket + 1] = value; - return storedValue; - } else { - return storedValue; - } - } else if (storedKey == EmptyKey) { - // Found an empty bucket. This means the key is not in the map. If we've already seen a deleted - // key, we should write at that position - if (firstDeletedKey != -1) { - bucket = firstDeletedKey; - } else { - ++usedBuckets; - } - - if (value == null) { - value = valueProvider.apply(key); - } - - table[bucket] = key; - table[bucket + 1] = value; - SIZE_UPDATER.incrementAndGet(this); - return valueProvider != null ? value : null; - } else if (storedKey == DeletedKey) { - // The bucket contained a different deleted key - if (firstDeletedKey == -1) { - firstDeletedKey = bucket; - } - } - - bucket = (bucket + ITEM_SIZE) & (table.length - 1); - } - } finally { - if (usedBuckets > resizeThresholdUp) { - try { - // Expand the hashmap - int newCapacity = alignToPowerOfTwo((int) (capacity * expandFactor)); - rehash(newCapacity); - } finally { - unlockWrite(stamp); - } - } else { - unlockWrite(stamp); - } - } - } - - private V remove(K key, Object value, int keyHash) { - long stamp = writeLock(); - int bucket = signSafeMod(keyHash, capacity); - - try { - while (true) { - K storedKey = (K) table[bucket]; - V storedValue = (V) table[bucket + 1]; - if (key.equals(storedKey)) { - if (value == null || value.equals(storedValue)) { - SIZE_UPDATER.decrementAndGet(this); - - int nextInArray = (bucket + ITEM_SIZE) & (table.length - 1); - if (table[nextInArray] == EmptyKey) { - table[bucket] = EmptyKey; - table[bucket + 1] = null; - --usedBuckets; - - // Cleanup all the buckets that were in `DeletedKey` state, - // so that we can reduce unnecessary expansions - int lastBucket = (bucket - ITEM_SIZE) & (table.length - 1); - while (table[lastBucket] == DeletedKey) { - table[lastBucket] = EmptyKey; - table[lastBucket + 1] = null; - --usedBuckets; - - lastBucket = (lastBucket - ITEM_SIZE) & (table.length - 1); - } - } else { - table[bucket] = DeletedKey; - table[bucket + 1] = null; - } - - return storedValue; - } else { - return null; - } - } else if (storedKey == EmptyKey) { - // Key wasn't found - return null; - } - - bucket = (bucket + ITEM_SIZE) & (table.length - 1); - } - - } finally { - if (autoShrink && size < resizeThresholdBelow) { - try { - // Shrinking must at least ensure initCapacity, - // so as to avoid frequent shrinking and expansion near initCapacity, - // frequent shrinking and expansion, - // additionally opened arrays will consume more memory and affect GC - int newCapacity = Math.max(alignToPowerOfTwo((int) (capacity / shrinkFactor)), initCapacity); - int newResizeThresholdUp = (int) (newCapacity * mapFillFactor); - if (newCapacity < capacity && newResizeThresholdUp > size) { - // shrink the hashmap - rehash(newCapacity); - } - } finally { - unlockWrite(stamp); - } - } else { - unlockWrite(stamp); - } - } - } - - void clear() { - long stamp = writeLock(); - - try { - if (autoShrink && capacity > initCapacity) { - shrinkToInitCapacity(); - } else { - Arrays.fill(table, EmptyKey); - this.size = 0; - this.usedBuckets = 0; - } - } finally { - unlockWrite(stamp); - } - } - - public void forEach(BiConsumer processor) { - // Take a reference to the data table, if there is a rehashing event, we'll be - // simply iterating over a snapshot of the data. - Object[] table = this.table; - - // Go through all the buckets for this section. We try to renew the stamp only after a validation - // error, otherwise we keep going with the same. - long stamp = 0; - for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) { - if (stamp == 0) { - stamp = tryOptimisticRead(); - } - - K storedKey = (K) table[bucket]; - V storedValue = (V) table[bucket + 1]; - - if (!validate(stamp)) { - // Fallback to acquiring read lock - stamp = readLock(); - - try { - storedKey = (K) table[bucket]; - storedValue = (V) table[bucket + 1]; - } finally { - unlockRead(stamp); - } - - stamp = 0; - } - - if (storedKey != DeletedKey && storedKey != EmptyKey) { - processor.accept(storedKey, storedValue); - } - } - } - - private void rehash(int newCapacity) { - // Expand the hashmap - Object[] newTable = new Object[ITEM_SIZE * newCapacity]; - - // Re-hash table - for (int i = 0; i < table.length; i += ITEM_SIZE) { - K storedKey = (K) table[i]; - V storedValue = (V) table[i + 1]; - if (storedKey != EmptyKey && storedKey != DeletedKey) { - insertKeyValueNoLock(newTable, newCapacity, storedKey, storedValue); - } - } - - table = newTable; - capacity = newCapacity; - usedBuckets = size; - resizeThresholdUp = (int) (capacity * mapFillFactor); - resizeThresholdBelow = (int) (capacity * mapIdleFactor); - } - - private void shrinkToInitCapacity() { - Object[] newTable = new Object[ITEM_SIZE * initCapacity]; - - table = newTable; - size = 0; - usedBuckets = 0; - // Capacity needs to be updated after the values, so that we won't see - // a capacity value bigger than the actual array size - capacity = initCapacity; - resizeThresholdUp = (int) (capacity * mapFillFactor); - resizeThresholdBelow = (int) (capacity * mapIdleFactor); - } - - private static void insertKeyValueNoLock(Object[] table, int capacity, K key, V value) { - int bucket = signSafeMod(hash(key), capacity); - - while (true) { - K storedKey = (K) table[bucket]; - - if (storedKey == EmptyKey) { - // The bucket is empty, so we can use it - table[bucket] = key; - table[bucket + 1] = value; - return; - } - - bucket = (bucket + ITEM_SIZE) & (table.length - 1); - } - } - } - - private static final long HashMixer = 0xc6a4a7935bd1e995L; - private static final int R = 47; - - static final long hash(K key) { - long hash = key.hashCode() * HashMixer; - hash ^= hash >>> R; - hash *= HashMixer; - return hash; - } - - static final int signSafeMod(long n, int max) { - // as the ITEM_SIZE of Section is 2, so the index is the multiple of 2 - // that is to left shift 1 bit - return (int) (n & (max - 1)) << 1; - } - - private static int alignToPowerOfTwo(int n) { - return (int) Math.pow(2, 32 - Integer.numberOfLeadingZeros(n - 1)); - } -} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java deleted file mode 100644 index 0a9f802037bce..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java +++ /dev/null @@ -1,622 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util.collections; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Objects.requireNonNull; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; -import java.util.concurrent.locks.StampedLock; -import java.util.function.Consumer; -import java.util.function.Predicate; - -/** - * Concurrent hash set. - * - *

Provides similar methods as a {@code ConcurrentMap} but since it's an open hash map with linear probing, - * no node allocations are required to store the values. - * - *
- * WARN: method forEach do not guarantee thread safety, nor does the values method. - *
- * The forEach method is specifically designed for single-threaded usage. When iterating over a set - * with concurrent writes, it becomes possible for new values to be either observed or not observed. - * There is no guarantee that if we write value1 and value2, and are able to see value2, then we will also see value1. - * - *
- * It is crucial to understand that the results obtained from aggregate status methods such as values - * are typically reliable only when the map is not undergoing concurrent updates from other threads. - * When concurrent updates are involved, the results of these methods reflect transient states - * that may be suitable for monitoring or estimation purposes, but not for program control. - * @param - */ -@SuppressWarnings("unchecked") -public class ConcurrentOpenHashSet { - - private static final Object EmptyValue = null; - private static final Object DeletedValue = new Object(); - - private static final int DefaultExpectedItems = 256; - private static final int DefaultConcurrencyLevel = 16; - - private static final float DefaultMapFillFactor = 0.66f; - private static final float DefaultMapIdleFactor = 0.15f; - - private static final float DefaultExpandFactor = 2; - private static final float DefaultShrinkFactor = 2; - - private static final boolean DefaultAutoShrink = false; - - private final Section[] sections; - - public static Builder newBuilder() { - return new Builder<>(); - } - - /** - * Builder of ConcurrentOpenHashSet. - */ - public static class Builder { - int expectedItems = DefaultExpectedItems; - int concurrencyLevel = DefaultConcurrencyLevel; - float mapFillFactor = DefaultMapFillFactor; - float mapIdleFactor = DefaultMapIdleFactor; - float expandFactor = DefaultExpandFactor; - float shrinkFactor = DefaultShrinkFactor; - boolean autoShrink = DefaultAutoShrink; - - public Builder expectedItems(int expectedItems) { - this.expectedItems = expectedItems; - return this; - } - - public Builder concurrencyLevel(int concurrencyLevel) { - this.concurrencyLevel = concurrencyLevel; - return this; - } - - public Builder mapFillFactor(float mapFillFactor) { - this.mapFillFactor = mapFillFactor; - return this; - } - - public Builder mapIdleFactor(float mapIdleFactor) { - this.mapIdleFactor = mapIdleFactor; - return this; - } - - public Builder expandFactor(float expandFactor) { - this.expandFactor = expandFactor; - return this; - } - - public Builder shrinkFactor(float shrinkFactor) { - this.shrinkFactor = shrinkFactor; - return this; - } - - public Builder autoShrink(boolean autoShrink) { - this.autoShrink = autoShrink; - return this; - } - - public ConcurrentOpenHashSet build() { - return new ConcurrentOpenHashSet<>(expectedItems, concurrencyLevel, - mapFillFactor, mapIdleFactor, autoShrink, expandFactor, shrinkFactor); - } - } - - @Deprecated - public ConcurrentOpenHashSet() { - this(DefaultExpectedItems); - } - - @Deprecated - public ConcurrentOpenHashSet(int expectedItems) { - this(expectedItems, DefaultConcurrencyLevel); - } - - @Deprecated - public ConcurrentOpenHashSet(int expectedItems, int concurrencyLevel) { - this(expectedItems, concurrencyLevel, DefaultMapFillFactor, DefaultMapIdleFactor, - DefaultAutoShrink, DefaultExpandFactor, DefaultShrinkFactor); - } - - public ConcurrentOpenHashSet(int expectedItems, int concurrencyLevel, - float mapFillFactor, float mapIdleFactor, - boolean autoShrink, float expandFactor, float shrinkFactor) { - checkArgument(expectedItems > 0); - checkArgument(concurrencyLevel > 0); - checkArgument(expectedItems >= concurrencyLevel); - checkArgument(mapFillFactor > 0 && mapFillFactor < 1); - checkArgument(mapIdleFactor > 0 && mapIdleFactor < 1); - checkArgument(mapFillFactor > mapIdleFactor); - checkArgument(expandFactor > 1); - checkArgument(shrinkFactor > 1); - - int numSections = concurrencyLevel; - int perSectionExpectedItems = expectedItems / numSections; - int perSectionCapacity = (int) (perSectionExpectedItems / mapFillFactor); - this.sections = (Section[]) new Section[numSections]; - - for (int i = 0; i < numSections; i++) { - sections[i] = new Section<>(perSectionCapacity, mapFillFactor, mapIdleFactor, - autoShrink, expandFactor, shrinkFactor); - } - } - - long getUsedBucketCount() { - long usedBucketCount = 0; - for (Section s : sections) { - usedBucketCount += s.usedBuckets; - } - return usedBucketCount; - } - - public long size() { - long size = 0; - for (int i = 0; i < sections.length; i++) { - size += sections[i].size; - } - return size; - } - - public long capacity() { - long capacity = 0; - for (int i = 0; i < sections.length; i++) { - capacity += sections[i].capacity; - } - return capacity; - } - - public boolean isEmpty() { - for (int i = 0; i < sections.length; i++) { - if (sections[i].size != 0) { - return false; - } - } - - return true; - } - - public boolean contains(V value) { - requireNonNull(value); - long h = hash(value); - return getSection(h).contains(value, (int) h); - } - - public boolean add(V value) { - requireNonNull(value); - long h = hash(value); - return getSection(h).add(value, (int) h); - } - - public boolean remove(V value) { - requireNonNull(value); - long h = hash(value); - return getSection(h).remove(value, (int) h); - } - - private Section getSection(long hash) { - // Use 32 msb out of long to get the section - final int sectionIdx = (int) (hash >>> 32) & (sections.length - 1); - return sections[sectionIdx]; - } - - public void clear() { - for (int i = 0; i < sections.length; i++) { - sections[i].clear(); - } - } - - /** - * Iterate over all the elements in the set and apply the provided function. - *

- * Warning: Do Not Guarantee Thread-Safety. - * @param processor the function to apply to each element - */ - public void forEach(Consumer processor) { - for (int i = 0; i < sections.length; i++) { - sections[i].forEach(processor); - } - } - - public int removeIf(Predicate filter) { - requireNonNull(filter); - - int removedCount = 0; - for (int i = 0; i < sections.length; i++) { - removedCount += sections[i].removeIf(filter); - } - - return removedCount; - } - - /** - * @return a new list of all values (makes a copy) - */ - public List values() { - List values = new ArrayList<>(); - forEach(value -> values.add(value)); - return values; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append('{'); - final AtomicBoolean first = new AtomicBoolean(true); - forEach(value -> { - if (!first.getAndSet(false)) { - sb.append(", "); - } - - sb.append(value.toString()); - }); - sb.append('}'); - return sb.toString(); - } - - // A section is a portion of the hash map that is covered by a single - @SuppressWarnings("serial") - private static final class Section extends StampedLock { - private volatile V[] values; - - private volatile int capacity; - private final int initCapacity; - private static final AtomicIntegerFieldUpdater

SIZE_UPDATER = - AtomicIntegerFieldUpdater.newUpdater(Section.class, "size"); - private volatile int size; - private int usedBuckets; - private int resizeThresholdUp; - private int resizeThresholdBelow; - private final float mapFillFactor; - private final float mapIdleFactor; - private final float expandFactor; - private final float shrinkFactor; - private final boolean autoShrink; - - Section(int capacity, float mapFillFactor, float mapIdleFactor, boolean autoShrink, - float expandFactor, float shrinkFactor) { - this.capacity = alignToPowerOfTwo(capacity); - this.initCapacity = this.capacity; - this.values = (V[]) new Object[this.capacity]; - this.size = 0; - this.usedBuckets = 0; - this.autoShrink = autoShrink; - this.mapFillFactor = mapFillFactor; - this.mapIdleFactor = mapIdleFactor; - this.expandFactor = expandFactor; - this.shrinkFactor = shrinkFactor; - this.resizeThresholdUp = (int) (this.capacity * mapFillFactor); - this.resizeThresholdBelow = (int) (this.capacity * mapIdleFactor); - } - - boolean contains(V value, int keyHash) { - long stamp = tryOptimisticRead(); - boolean acquiredLock = false; - - // add local variable here, so OutOfBound won't happen - V[] values = this.values; - // calculate table.length as capacity to avoid rehash changing capacity - int bucket = signSafeMod(keyHash, values.length); - - try { - while (true) { - // First try optimistic locking - V storedValue = values[bucket]; - - if (!acquiredLock && validate(stamp)) { - // The values we have read are consistent - if (value.equals(storedValue)) { - return true; - } else if (storedValue == EmptyValue) { - // Not found - return false; - } - } else { - // Fallback to acquiring read lock - if (!acquiredLock) { - stamp = readLock(); - acquiredLock = true; - - // update local variable - values = this.values; - bucket = signSafeMod(keyHash, values.length); - storedValue = values[bucket]; - } - - if (value.equals(storedValue)) { - return true; - } else if (storedValue == EmptyValue) { - // Not found - return false; - } - } - bucket = (bucket + 1) & (values.length - 1); - } - } finally { - if (acquiredLock) { - unlockRead(stamp); - } - } - } - - boolean add(V value, int keyHash) { - int bucket = keyHash; - - long stamp = writeLock(); - int capacity = this.capacity; - - // Remember where we find the first available spot - int firstDeletedValue = -1; - - try { - while (true) { - bucket = signSafeMod(bucket, capacity); - - V storedValue = values[bucket]; - - if (value.equals(storedValue)) { - return false; - } else if (storedValue == EmptyValue) { - // Found an empty bucket. This means the value is not in the set. If we've already seen a - // deleted value, we should write at that position - if (firstDeletedValue != -1) { - bucket = firstDeletedValue; - } else { - ++usedBuckets; - } - - values[bucket] = value; - SIZE_UPDATER.incrementAndGet(this); - return true; - } else if (storedValue == DeletedValue) { - // The bucket contained a different deleted key - if (firstDeletedValue == -1) { - firstDeletedValue = bucket; - } - } - - ++bucket; - } - } finally { - if (usedBuckets > resizeThresholdUp) { - try { - // Expand the hashmap - int newCapacity = alignToPowerOfTwo((int) (capacity * expandFactor)); - rehash(newCapacity); - } finally { - unlockWrite(stamp); - } - } else { - unlockWrite(stamp); - } - } - } - - private boolean remove(V value, int keyHash) { - int bucket = keyHash; - long stamp = writeLock(); - - try { - while (true) { - int capacity = this.capacity; - bucket = signSafeMod(bucket, capacity); - - V storedValue = values[bucket]; - if (value.equals(storedValue)) { - SIZE_UPDATER.decrementAndGet(this); - cleanBucket(bucket); - return true; - } else if (storedValue == EmptyValue) { - // Value wasn't found - return false; - } - - ++bucket; - } - - } finally { - if (autoShrink && size < resizeThresholdBelow) { - try { - // Shrinking must at least ensure initCapacity, - // so as to avoid frequent shrinking and expansion near initCapacity, - // frequent shrinking and expansion, - // additionally opened arrays will consume more memory and affect GC - int newCapacity = Math.max(alignToPowerOfTwo((int) (capacity / shrinkFactor)), initCapacity); - int newResizeThresholdUp = (int) (newCapacity * mapFillFactor); - if (newCapacity < capacity && newResizeThresholdUp > size) { - // shrink the hashmap - rehash(newCapacity); - } - } finally { - unlockWrite(stamp); - } - } else { - unlockWrite(stamp); - } - } - } - - void clear() { - long stamp = writeLock(); - - try { - if (autoShrink && capacity > initCapacity) { - shrinkToInitCapacity(); - } else { - Arrays.fill(values, EmptyValue); - this.size = 0; - this.usedBuckets = 0; - } - } finally { - unlockWrite(stamp); - } - } - - int removeIf(Predicate filter) { - long stamp = writeLock(); - - int removedCount = 0; - try { - // Go through all the buckets for this section - for (int bucket = capacity - 1; bucket >= 0; bucket--) { - V storedValue = values[bucket]; - - if (storedValue != DeletedValue && storedValue != EmptyValue) { - if (filter.test(storedValue)) { - // Removing item - SIZE_UPDATER.decrementAndGet(this); - ++removedCount; - cleanBucket(bucket); - } - } - } - - return removedCount; - } finally { - unlockWrite(stamp); - } - } - - private void cleanBucket(int bucket) { - int nextInArray = signSafeMod(bucket + 1, capacity); - if (values[nextInArray] == EmptyValue) { - values[bucket] = (V) EmptyValue; - --usedBuckets; - - // Cleanup all the buckets that were in `DeletedValue` state, - // so that we can reduce unnecessary expansions - int lastBucket = signSafeMod(bucket - 1, capacity); - while (values[lastBucket] == DeletedValue) { - values[lastBucket] = (V) EmptyValue; - --usedBuckets; - - lastBucket = signSafeMod(lastBucket - 1, capacity); - } - } else { - values[bucket] = (V) DeletedValue; - } - } - - public void forEach(Consumer processor) { - V[] values = this.values; - - // Go through all the buckets for this section. We try to renew the stamp only after a validation - // error, otherwise we keep going with the same. - long stamp = 0; - for (int bucket = 0; bucket < capacity; bucket++) { - if (stamp == 0) { - stamp = tryOptimisticRead(); - } - - V storedValue = values[bucket]; - - if (!validate(stamp)) { - // Fallback to acquiring read lock - stamp = readLock(); - - try { - storedValue = values[bucket]; - } finally { - unlockRead(stamp); - } - - stamp = 0; - } - - if (storedValue != DeletedValue && storedValue != EmptyValue) { - processor.accept(storedValue); - } - } - } - - private void rehash(int newCapacity) { - // Expand the hashmap - V[] newValues = (V[]) new Object[newCapacity]; - - // Re-hash table - for (int i = 0; i < values.length; i++) { - V storedValue = values[i]; - if (storedValue != EmptyValue && storedValue != DeletedValue) { - insertValueNoLock(newValues, storedValue); - } - } - - values = newValues; - capacity = newCapacity; - usedBuckets = size; - resizeThresholdUp = (int) (capacity * mapFillFactor); - resizeThresholdBelow = (int) (capacity * mapIdleFactor); - } - - private void shrinkToInitCapacity() { - V[] newValues = (V[]) new Object[initCapacity]; - - values = newValues; - size = 0; - usedBuckets = 0; - // Capacity needs to be updated after the values, so that we won't see - // a capacity value bigger than the actual array size - capacity = initCapacity; - resizeThresholdUp = (int) (capacity * mapFillFactor); - resizeThresholdBelow = (int) (capacity * mapIdleFactor); - } - - private static void insertValueNoLock(V[] values, V value) { - int bucket = (int) hash(value); - - while (true) { - bucket = signSafeMod(bucket, values.length); - - V storedValue = values[bucket]; - - if (storedValue == EmptyValue) { - // The bucket is empty, so we can use it - values[bucket] = value; - return; - } - - ++bucket; - } - } - } - - private static final long HashMixer = 0xc6a4a7935bd1e995L; - private static final int R = 47; - - static final long hash(K key) { - long hash = key.hashCode() * HashMixer; - hash ^= hash >>> R; - hash *= HashMixer; - return hash; - } - - static final int signSafeMod(long n, int max) { - return (int) n & (max - 1); - } - - private static int alignToPowerOfTwo(int n) { - return (int) Math.pow(2, 32 - Integer.numberOfLeadingZeros(n - 1)); - } -} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java deleted file mode 100644 index 48a1a705a3202..0000000000000 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java +++ /dev/null @@ -1,700 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util.collections; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertThrows; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Random; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; - -import lombok.Cleanup; -import org.testng.annotations.Test; - -import com.google.common.collect.Lists; - -public class ConcurrentOpenHashMapTest { - - @Test - public void testConstructor() { - try { - ConcurrentOpenHashMap.newBuilder() - .expectedItems(0) - .build(); - fail("should have thrown exception"); - } catch (IllegalArgumentException e) { - // ok - } - - try { - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(0) - .build(); - fail("should have thrown exception"); - } catch (IllegalArgumentException e) { - // ok - } - - try { - ConcurrentOpenHashMap.newBuilder() - .expectedItems(4) - .concurrencyLevel(8) - .build(); - fail("should have thrown exception"); - } catch (IllegalArgumentException e) { - // ok - } - } - - @Test - public void simpleInsertions() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .build(); - - assertTrue(map.isEmpty()); - assertNull(map.put("1", "one")); - assertFalse(map.isEmpty()); - - assertNull(map.put("2", "two")); - assertNull(map.put("3", "three")); - - assertEquals(map.size(), 3); - - assertEquals(map.get("1"), "one"); - assertEquals(map.size(), 3); - - assertEquals(map.remove("1"), "one"); - assertEquals(map.size(), 2); - assertNull(map.get("1")); - assertNull(map.get("5")); - assertEquals(map.size(), 2); - - assertNull(map.put("1", "one")); - assertEquals(map.size(), 3); - assertEquals(map.put("1", "uno"), "one"); - assertEquals(map.size(), 3); - } - - @Test - public void testReduceUnnecessaryExpansions() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .build(); - assertNull(map.put("1", "1")); - assertNull(map.put("2", "2")); - assertNull(map.put("3", "3")); - assertNull(map.put("4", "4")); - - assertEquals(map.remove("1"), "1"); - assertEquals(map.remove("2"), "2"); - assertEquals(map.remove("3"), "3"); - assertEquals(map.remove("4"), "4"); - - assertEquals(0, map.getUsedBucketCount()); - } - - @Test - public void testClear() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertTrue(map.capacity() == 4); - - assertNull(map.put("k1", "v1")); - assertNull(map.put("k2", "v2")); - assertNull(map.put("k3", "v3")); - - assertTrue(map.capacity() == 8); - map.clear(); - assertTrue(map.capacity() == 4); - } - - @Test - public void testExpandAndShrink() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertTrue(map.capacity() == 4); - - assertNull(map.put("k1", "v1")); - assertNull(map.put("k2", "v2")); - assertNull(map.put("k3", "v3")); - - // expand hashmap - assertTrue(map.capacity() == 8); - - assertTrue(map.remove("k1", "v1")); - // not shrink - assertTrue(map.capacity() == 8); - assertTrue(map.remove("k2", "v2")); - // shrink hashmap - assertTrue(map.capacity() == 4); - - // expand hashmap - assertNull(map.put("k4", "v4")); - assertNull(map.put("k5", "v5")); - assertTrue(map.capacity() == 8); - - //verify that the map does not keep shrinking at every remove() operation - assertNull(map.put("k6", "v6")); - assertTrue(map.remove("k6", "v6")); - assertTrue(map.capacity() == 8); - } - - @Test - public void testExpandShrinkAndClear() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - final long initCapacity = map.capacity(); - assertTrue(map.capacity() == 4); - assertNull(map.put("k1", "v1")); - assertNull(map.put("k2", "v2")); - assertNull(map.put("k3", "v3")); - - // expand hashmap - assertTrue(map.capacity() == 8); - - assertTrue(map.remove("k1", "v1")); - // not shrink - assertTrue(map.capacity() == 8); - assertTrue(map.remove("k2", "v2")); - // shrink hashmap - assertTrue(map.capacity() == 4); - - assertTrue(map.remove("k3", "v3")); - // Will not shrink the hashmap again because shrink capacity is less than initCapacity - // current capacity is equal than the initial capacity - assertTrue(map.capacity() == initCapacity); - map.clear(); - // after clear, because current capacity is equal than the initial capacity, so not shrinkToInitCapacity - assertTrue(map.capacity() == initCapacity); - } - - @Test - public void testConcurrentExpandAndShrinkAndGet() throws Throwable { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertEquals(map.capacity(), 4); - - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - final int readThreads = 16; - final int writeThreads = 1; - final int n = 1_000; - CyclicBarrier barrier = new CyclicBarrier(writeThreads + readThreads); - Future future = null; - AtomicReference ex = new AtomicReference<>(); - - for (int i = 0; i < readThreads; i++) { - executor.submit(() -> { - try { - barrier.await(); - } catch (Exception e) { - throw new RuntimeException(e); - } - while (!Thread.currentThread().isInterrupted()) { - try { - map.get("k2"); - } catch (Exception e) { - ex.set(e); - } - } - }); - } - - assertNull(map.put("k1","v1")); - future = executor.submit(() -> { - try { - barrier.await(); - } catch (Exception e) { - throw new RuntimeException(e); - } - - for (int i = 0; i < n; i++) { - // expand hashmap - assertNull(map.put("k2", "v2")); - assertNull(map.put("k3", "v3")); - assertEquals(map.capacity(), 8); - - // shrink hashmap - assertTrue(map.remove("k2", "v2")); - assertTrue(map.remove("k3", "v3")); - assertEquals(map.capacity(), 4); - } - }); - - future.get(); - assertTrue(ex.get() == null); - // shut down pool - executor.shutdown(); - } - - @Test - public void testRemove() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - - assertTrue(map.isEmpty()); - assertNull(map.put("1", "one")); - assertFalse(map.isEmpty()); - - assertFalse(map.remove("0", "zero")); - assertFalse(map.remove("1", "uno")); - - assertFalse(map.isEmpty()); - assertTrue(map.remove("1", "one")); - assertTrue(map.isEmpty()); - } - - @Test - public void testRehashing() { - int n = 16; - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(n / 2) - .concurrencyLevel(1) - .build(); - assertEquals(map.capacity(), n); - assertEquals(map.size(), 0); - - for (int i = 0; i < n; i++) { - map.put(Integer.toString(i), i); - } - - assertEquals(map.capacity(), 2 * n); - assertEquals(map.size(), n); - } - - @Test - public void testRehashingWithDeletes() { - int n = 16; - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(n / 2) - .concurrencyLevel(1) - .build(); - assertEquals(map.capacity(), n); - assertEquals(map.size(), 0); - - for (int i = 0; i < n / 2; i++) { - map.put(i, i); - } - - for (int i = 0; i < n / 2; i++) { - map.remove(i); - } - - for (int i = n; i < (2 * n); i++) { - map.put(i, i); - } - - assertEquals(map.capacity(), 2 * n); - assertEquals(map.size(), n); - } - - @Test - public void concurrentInsertions() throws Throwable { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - - final int nThreads = 16; - final int N = 100_000; - String value = "value"; - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - final int threadIdx = i; - - futures.add(executor.submit(() -> { - Random random = new Random(); - - for (int j = 0; j < N; j++) { - long key = random.nextLong(); - // Ensure keys are uniques - key -= key % (threadIdx + 1); - - map.put(key, value); - } - })); - } - - for (Future future : futures) { - future.get(); - } - - assertEquals(map.size(), N * nThreads); - } - - @Test - public void concurrentInsertionsAndReads() throws Throwable { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - - final int nThreads = 16; - final int N = 100_000; - String value = "value"; - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - final int threadIdx = i; - - futures.add(executor.submit(() -> { - Random random = new Random(); - - for (int j = 0; j < N; j++) { - long key = random.nextLong(); - // Ensure keys are uniques - key -= key % (threadIdx + 1); - - map.put(key, value); - } - })); - } - - for (Future future : futures) { - future.get(); - } - - assertEquals(map.size(), N * nThreads); - } - - @Test - public void testIteration() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - - assertEquals(map.keys(), Collections.emptyList()); - assertEquals(map.values(), Collections.emptyList()); - - map.put(0l, "zero"); - - assertEquals(map.keys(), Lists.newArrayList(0l)); - assertEquals(map.values(), Lists.newArrayList("zero")); - - map.remove(0l); - - assertEquals(map.keys(), Collections.emptyList()); - assertEquals(map.values(), Collections.emptyList()); - - map.put(0l, "zero"); - map.put(1l, "one"); - map.put(2l, "two"); - - List keys = map.keys(); - keys.sort(null); - assertEquals(keys, Lists.newArrayList(0l, 1l, 2l)); - - List values = map.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList("one", "two", "zero")); - - map.put(1l, "uno"); - - keys = map.keys(); - keys.sort(null); - assertEquals(keys, Lists.newArrayList(0l, 1l, 2l)); - - values = map.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList("two", "uno", "zero")); - - map.clear(); - assertTrue(map.isEmpty()); - } - - @Test - public void testHashConflictWithDeletion() { - final int Buckets = 16; - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(Buckets) - .concurrencyLevel(1) - .build(); - - // Pick 2 keys that fall into the same bucket - long key1 = 1; - long key2 = 27; - - int bucket1 = ConcurrentOpenHashMap.signSafeMod(ConcurrentOpenHashMap.hash(key1), Buckets); - int bucket2 = ConcurrentOpenHashMap.signSafeMod(ConcurrentOpenHashMap.hash(key2), Buckets); - assertEquals(bucket1, bucket2); - - assertNull(map.put(key1, "value-1")); - assertNull(map.put(key2, "value-2")); - assertEquals(map.size(), 2); - - assertEquals(map.remove(key1), "value-1"); - assertEquals(map.size(), 1); - - assertNull(map.put(key1, "value-1-overwrite")); - assertEquals(map.size(), 2); - - assertEquals(map.remove(key1), "value-1-overwrite"); - assertEquals(map.size(), 1); - - assertEquals(map.put(key2, "value-2-overwrite"), "value-2"); - assertEquals(map.get(key2), "value-2-overwrite"); - - assertEquals(map.size(), 1); - assertEquals(map.remove(key2), "value-2-overwrite"); - assertTrue(map.isEmpty()); - } - - @Test - public void testPutIfAbsent() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - assertNull(map.putIfAbsent(1l, "one")); - assertEquals(map.get(1l), "one"); - - assertEquals(map.putIfAbsent(1l, "uno"), "one"); - assertEquals(map.get(1l), "one"); - } - - @Test - public void testComputeIfAbsent() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - AtomicInteger counter = new AtomicInteger(); - Function provider = key -> counter.getAndIncrement(); - - assertEquals(map.computeIfAbsent(0, provider).intValue(), 0); - assertEquals(map.get(0).intValue(), 0); - - assertEquals(map.computeIfAbsent(1, provider).intValue(), 1); - assertEquals(map.get(1).intValue(), 1); - - assertEquals(map.computeIfAbsent(1, provider).intValue(), 1); - assertEquals(map.get(1).intValue(), 1); - - assertEquals(map.computeIfAbsent(2, provider).intValue(), 2); - assertEquals(map.get(2).intValue(), 2); - } - - @Test - public void testEqualsKeys() { - class T { - int value; - - T(int value) { - this.value = value; - } - - @Override - public int hashCode() { - return Integer.hashCode(value); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof T) { - return value == ((T) obj).value; - } - - return false; - } - } - - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - - T t1 = new T(1); - T t1_b = new T(1); - T t2 = new T(2); - - assertEquals(t1, t1_b); - assertNotEquals(t2, t1); - assertNotEquals(t2, t1_b); - - assertNull(map.put(t1, "t1")); - assertEquals(map.get(t1), "t1"); - assertEquals(map.get(t1_b), "t1"); - assertNull(map.get(t2)); - - assertEquals(map.remove(t1_b), "t1"); - assertNull(map.get(t1)); - assertNull(map.get(t1_b)); - } - - @Test - public void testNullValue() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - String key = "a"; - assertThrows(NullPointerException.class, () -> map.put(key, null)); - - //put a null value. - assertNull(map.computeIfAbsent(key, k -> null)); - assertEquals(1, map.size()); - assertEquals(1, map.keys().size()); - assertEquals(1, map.values().size()); - assertNull(map.get(key)); - assertFalse(map.containsKey(key)); - - //test remove null value - map.removeNullValue(key); - assertTrue(map.isEmpty()); - assertEquals(0, map.keys().size()); - assertEquals(0, map.values().size()); - assertNull(map.get(key)); - assertFalse(map.containsKey(key)); - - - //test not remove non-null value - map.put(key, "V"); - assertEquals(1, map.size()); - map.removeNullValue(key); - assertEquals(1, map.size()); - - } - - static final int Iterations = 1; - static final int ReadIterations = 1000; - static final int N = 1_000_000; - - public void benchConcurrentOpenHashMap() throws Exception { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(N) - .concurrencyLevel(1) - .build(); - - for (long i = 0; i < Iterations; i++) { - for (int j = 0; j < N; j++) { - map.put(i, "value"); - } - - for (long h = 0; h < ReadIterations; h++) { - for (int j = 0; j < N; j++) { - map.get(i); - } - } - - for (long j = 0; j < N; j++) { - map.remove(i); - } - } - } - - public void benchConcurrentHashMap() throws Exception { - ConcurrentHashMap map = new ConcurrentHashMap(N, 0.66f, 1); - - for (long i = 0; i < Iterations; i++) { - for (int j = 0; j < N; j++) { - map.put(i, "value"); - } - - for (long h = 0; h < ReadIterations; h++) { - for (int j = 0; j < N; j++) { - map.get(i); - } - } - - for (int j = 0; j < N; j++) { - map.remove(i); - } - } - } - - void benchHashMap() { - HashMap map = new HashMap<>(N, 0.66f); - - for (long i = 0; i < Iterations; i++) { - for (int j = 0; j < N; j++) { - map.put(i, "value"); - } - - for (long h = 0; h < ReadIterations; h++) { - for (int j = 0; j < N; j++) { - map.get(i); - } - } - - for (int j = 0; j < N; j++) { - map.remove(i); - } - } - } - - public static void main(String[] args) throws Exception { - ConcurrentOpenHashMapTest t = new ConcurrentOpenHashMapTest(); - - long start = System.nanoTime(); - t.benchHashMap(); - long end = System.nanoTime(); - - System.out.println("HM: " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms"); - - start = System.nanoTime(); - t.benchConcurrentHashMap(); - end = System.nanoTime(); - - System.out.println("CHM: " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms"); - - start = System.nanoTime(); - t.benchConcurrentOpenHashMap(); - end = System.nanoTime(); - - System.out.println("CLHM: " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms"); - - } -} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java deleted file mode 100644 index d509002e21998..0000000000000 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java +++ /dev/null @@ -1,503 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.common.util.collections; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertThrows; -import static org.testng.Assert.assertTrue; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Random; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicReference; - -import lombok.Cleanup; -import org.testng.annotations.Test; - -import com.google.common.collect.Lists; - -// Deprecation warning suppressed as this test targets deprecated class -@SuppressWarnings("deprecation") -public class ConcurrentOpenHashSetTest { - - @Test - public void testConstructor() { - assertThrows(IllegalArgumentException.class, () -> new ConcurrentOpenHashSet(0)); - assertThrows(IllegalArgumentException.class, () -> new ConcurrentOpenHashSet(16, 0)); - assertThrows(IllegalArgumentException.class, () -> new ConcurrentOpenHashSet(4, 8)); - } - - @Test - public void simpleInsertions() { - ConcurrentOpenHashSet set = new ConcurrentOpenHashSet<>(16); - - assertTrue(set.isEmpty()); - assertTrue(set.add("1")); - assertFalse(set.isEmpty()); - - assertTrue(set.add("2")); - assertTrue(set.add("3")); - - assertEquals(set.size(), 3); - - assertTrue(set.contains("1")); - assertEquals(set.size(), 3); - - assertTrue(set.remove("1")); - assertEquals(set.size(), 2); - assertFalse(set.contains("1")); - assertFalse(set.contains("5")); - assertEquals(set.size(), 2); - - assertTrue(set.add("1")); - assertEquals(set.size(), 3); - assertFalse(set.add("1")); - assertEquals(set.size(), 3); - } - - @Test - public void testReduceUnnecessaryExpansions() { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .build(); - - assertTrue(set.add("1")); - assertTrue(set.add("2")); - assertTrue(set.add("3")); - assertTrue(set.add("4")); - - assertTrue(set.remove("1")); - assertTrue(set.remove("2")); - assertTrue(set.remove("3")); - assertTrue(set.remove("4")); - assertEquals(0, set.getUsedBucketCount()); - } - - @Test - public void testClear() { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertEquals(set.capacity(), 4); - - assertTrue(set.add("k1")); - assertTrue(set.add("k2")); - assertTrue(set.add("k3")); - - assertEquals(set.capacity(), 8); - set.clear(); - assertEquals(set.capacity(), 4); - } - - @Test - public void testExpandAndShrink() { - ConcurrentOpenHashSet map = - ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertEquals(map.capacity(), 4); - - assertTrue(map.add("k1")); - assertTrue(map.add("k2")); - assertTrue(map.add("k3")); - - // expand hashmap - assertEquals(map.capacity(), 8); - - assertTrue(map.remove("k1")); - // not shrink - assertEquals(map.capacity(), 8); - assertTrue(map.remove("k2")); - // shrink hashmap - assertEquals(map.capacity(), 4); - - // expand hashmap - assertTrue(map.add("k4")); - assertTrue(map.add("k5")); - assertEquals(map.capacity(), 8); - - //verify that the map does not keep shrinking at every remove() operation - assertTrue(map.add("k6")); - assertTrue(map.remove("k6")); - assertEquals(map.capacity(), 8); - } - - @Test - public void testExpandShrinkAndClear() { - ConcurrentOpenHashSet map = ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - final long initCapacity = map.capacity(); - assertTrue(map.capacity() == 4); - - assertTrue(map.add("k1")); - assertTrue(map.add("k2")); - assertTrue(map.add("k3")); - - // expand hashmap - assertTrue(map.capacity() == 8); - - assertTrue(map.remove("k1")); - // not shrink - assertTrue(map.capacity() == 8); - assertTrue(map.remove("k2")); - // shrink hashmap - assertTrue(map.capacity() == 4); - - assertTrue(map.remove("k3")); - // Will not shrink the hashmap again because shrink capacity is less than initCapacity - // current capacity is equal than the initial capacity - assertTrue(map.capacity() == initCapacity); - map.clear(); - // after clear, because current capacity is equal than the initial capacity, so not shrinkToInitCapacity - assertTrue(map.capacity() == initCapacity); - } - - @Test - public void testConcurrentExpandAndShrinkAndGet() throws Throwable { - ConcurrentOpenHashSet set = ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertEquals(set.capacity(), 4); - - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - final int readThreads = 16; - final int writeThreads = 1; - final int n = 1_000; - CyclicBarrier barrier = new CyclicBarrier(writeThreads + readThreads); - Future future = null; - AtomicReference ex = new AtomicReference<>(); - - for (int i = 0; i < readThreads; i++) { - executor.submit(() -> { - try { - barrier.await(); - } catch (Exception e) { - throw new RuntimeException(e); - } - while (!Thread.currentThread().isInterrupted()) { - try { - set.contains("k2"); - } catch (Exception e) { - ex.set(e); - } - } - }); - } - - assertTrue(set.add("k1")); - future = executor.submit(() -> { - try { - barrier.await(); - } catch (Exception e) { - throw new RuntimeException(e); - } - - for (int i = 0; i < n; i++) { - // expand hashmap - assertTrue(set.add("k2")); - assertTrue(set.add("k3")); - assertEquals(set.capacity(), 8); - - // shrink hashmap - assertTrue(set.remove("k2")); - assertTrue(set.remove("k3")); - assertEquals(set.capacity(), 4); - } - }); - - future.get(); - assertTrue(ex.get() == null); - // shut down pool - executor.shutdown(); - } - - @Test - public void testRemove() { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder().build(); - - assertTrue(set.isEmpty()); - assertTrue(set.add("1")); - assertFalse(set.isEmpty()); - - assertFalse(set.remove("0")); - assertFalse(set.isEmpty()); - assertTrue(set.remove("1")); - assertTrue(set.isEmpty()); - } - - @Test - public void testRehashing() { - int n = 16; - ConcurrentOpenHashSet set = new ConcurrentOpenHashSet<>(n / 2, 1); - assertEquals(set.capacity(), n); - assertEquals(set.size(), 0); - - for (int i = 0; i < n; i++) { - set.add(i); - } - - assertEquals(set.capacity(), 2 * n); - assertEquals(set.size(), n); - } - - @Test - public void testRehashingWithDeletes() { - int n = 16; - ConcurrentOpenHashSet set = new ConcurrentOpenHashSet<>(n / 2, 1); - assertEquals(set.capacity(), n); - assertEquals(set.size(), 0); - - for (int i = 0; i < n / 2; i++) { - set.add(i); - } - - for (int i = 0; i < n / 2; i++) { - set.remove(i); - } - - for (int i = n; i < (2 * n); i++) { - set.add(i); - } - - assertEquals(set.capacity(), 2 * n); - assertEquals(set.size(), n); - } - - @Test - public void concurrentInsertions() throws Throwable { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder().build(); - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - - final int nThreads = 16; - final int N = 100_000; - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - final int threadIdx = i; - - futures.add(executor.submit(() -> { - Random random = new Random(); - - for (int j = 0; j < N; j++) { - long key = random.nextLong(); - // Ensure keys are unique - key -= key % (threadIdx + 1); - - set.add(key); - } - })); - } - - for (Future future : futures) { - future.get(); - } - - assertEquals(set.size(), N * nThreads); - } - - @Test - public void concurrentInsertionsAndReads() throws Throwable { - ConcurrentOpenHashSet map = - ConcurrentOpenHashSet.newBuilder().build(); - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - - final int nThreads = 16; - final int N = 100_000; - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - final int threadIdx = i; - - futures.add(executor.submit(() -> { - Random random = new Random(); - - for (int j = 0; j < N; j++) { - long key = random.nextLong(); - // Ensure keys are unique - key -= key % (threadIdx + 1); - - map.add(key); - } - })); - } - - for (Future future : futures) { - future.get(); - } - - assertEquals(map.size(), N * nThreads); - } - - @Test - public void testIteration() { - ConcurrentOpenHashSet set = ConcurrentOpenHashSet.newBuilder().build(); - - assertEquals(set.values(), Collections.emptyList()); - - set.add(0l); - - assertEquals(set.values(), Lists.newArrayList(0l)); - - set.remove(0l); - - assertEquals(set.values(), Collections.emptyList()); - - set.add(0l); - set.add(1l); - set.add(2l); - - List values = set.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList(0l, 1l, 2l)); - - set.clear(); - assertTrue(set.isEmpty()); - } - - @Test - public void testRemoval() { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder().build(); - - set.add(0); - set.add(1); - set.add(3); - set.add(6); - set.add(7); - - List values = set.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList(0, 1, 3, 6, 7)); - - int numOfItemsDeleted = set.removeIf(i -> i < 5); - assertEquals(numOfItemsDeleted, 3); - assertEquals(set.size(), values.size() - numOfItemsDeleted); - values = set.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList(6, 7)); - } - - @Test - public void testHashConflictWithDeletion() { - final int Buckets = 16; - ConcurrentOpenHashSet set = new ConcurrentOpenHashSet<>(Buckets, 1); - - // Pick 2 keys that fall into the same bucket - long key1 = 1; - long key2 = 27; - - int bucket1 = ConcurrentOpenHashSet.signSafeMod(ConcurrentOpenHashSet.hash(key1), Buckets); - int bucket2 = ConcurrentOpenHashSet.signSafeMod(ConcurrentOpenHashSet.hash(key2), Buckets); - assertEquals(bucket1, bucket2); - - assertTrue(set.add(key1)); - assertTrue(set.add(key2)); - assertEquals(set.size(), 2); - - assertTrue(set.remove(key1)); - assertEquals(set.size(), 1); - - assertTrue(set.add(key1)); - assertEquals(set.size(), 2); - - assertTrue(set.remove(key1)); - assertEquals(set.size(), 1); - - assertFalse(set.add(key2)); - assertTrue(set.contains(key2)); - - assertEquals(set.size(), 1); - assertTrue(set.remove(key2)); - assertTrue(set.isEmpty()); - } - - @Test - public void testEqualsObjects() { - class T { - int value; - - T(int value) { - this.value = value; - } - - @Override - public int hashCode() { - return Integer.hashCode(value); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof T) { - return value == ((T) obj).value; - } - - return false; - } - } - - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder().build(); - - T t1 = new T(1); - T t1_b = new T(1); - T t2 = new T(2); - - assertEquals(t1, t1_b); - assertNotEquals(t2, t1); - assertNotEquals(t2, t1_b); - - set.add(t1); - assertTrue(set.contains(t1)); - assertTrue(set.contains(t1_b)); - assertFalse(set.contains(t2)); - - assertTrue(set.remove(t1_b)); - assertFalse(set.contains(t1)); - assertFalse(set.contains(t1_b)); - } - -} diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java index 889f4431cc35b..7bb4df7baa533 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java @@ -23,7 +23,10 @@ import java.io.Closeable; import java.io.IOException; import java.net.MalformedURLException; +import java.util.Map; import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -44,8 +47,6 @@ import org.apache.pulsar.client.internal.PropertiesUtils; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -73,9 +74,9 @@ public class WebSocketService implements Closeable { private Optional cryptoKeyReader = Optional.empty(); private ClusterData localCluster; - private final ConcurrentOpenHashMap> topicProducerMap; - private final ConcurrentOpenHashMap> topicConsumerMap; - private final ConcurrentOpenHashMap> topicReaderMap; + private final Map> topicProducerMap = new ConcurrentHashMap<>(); + private final Map> topicConsumerMap = new ConcurrentHashMap<>(); + private final Map> topicReaderMap = new ConcurrentHashMap<>(); private final ProxyStats proxyStats; public WebSocketService(WebSocketProxyConfiguration config) { @@ -88,17 +89,6 @@ public WebSocketService(ClusterData localCluster, ServiceConfiguration config) { .newScheduledThreadPool(config.getWebSocketNumServiceThreads(), new DefaultThreadFactory("pulsar-websocket")); this.localCluster = localCluster; - this.topicProducerMap = - ConcurrentOpenHashMap.>newBuilder() - .build(); - this.topicConsumerMap = - ConcurrentOpenHashMap.>newBuilder() - .build(); - this.topicReaderMap = - ConcurrentOpenHashMap.>newBuilder() - .build(); this.proxyStats = new ProxyStats(this); } @@ -288,11 +278,11 @@ public boolean isAuthorizationEnabled() { public boolean addProducer(ProducerHandler producer) { return topicProducerMap .computeIfAbsent(producer.getProducer().getTopic(), - topic -> ConcurrentOpenHashSet.newBuilder().build()) + topic -> ConcurrentHashMap.newKeySet()) .add(producer); } - public ConcurrentOpenHashMap> getProducers() { + public Map> getProducers() { return topicProducerMap; } @@ -306,12 +296,11 @@ public boolean removeProducer(ProducerHandler producer) { public boolean addConsumer(ConsumerHandler consumer) { return topicConsumerMap - .computeIfAbsent(consumer.getConsumer().getTopic(), topic -> - ConcurrentOpenHashSet.newBuilder().build()) + .computeIfAbsent(consumer.getConsumer().getTopic(), topic -> ConcurrentHashMap.newKeySet()) .add(consumer); } - public ConcurrentOpenHashMap> getConsumers() { + public Map> getConsumers() { return topicConsumerMap; } @@ -324,12 +313,11 @@ public boolean removeConsumer(ConsumerHandler consumer) { } public boolean addReader(ReaderHandler reader) { - return topicReaderMap.computeIfAbsent(reader.getConsumer().getTopic(), topic -> - ConcurrentOpenHashSet.newBuilder().build()) + return topicReaderMap.computeIfAbsent(reader.getConsumer().getTopic(), topic -> ConcurrentHashMap.newKeySet()) .add(reader); } - public ConcurrentOpenHashMap> getReaders() { + public Map> getReaders() { return topicReaderMap; } diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyStats.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyStats.java index eb1566ef7d412..4660340e9cc54 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyStats.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyStats.java @@ -24,11 +24,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.JvmMetrics; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.websocket.WebSocketService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,7 +41,7 @@ public class ProxyStats { private final WebSocketService service; private final JvmMetrics jvmMetrics; - private ConcurrentOpenHashMap topicStats; + private final Map topicStats = new ConcurrentHashMap<>(); private List metricsCollection; private List tempMetricsCollection; @@ -50,9 +50,6 @@ public ProxyStats(WebSocketService service) { this.service = service; this.jvmMetrics = JvmMetrics.create( service.getExecutor(), "prx", service.getConfig().getJvmGCMetricsLoggerClassName()); - this.topicStats = - ConcurrentOpenHashMap.newBuilder() - .build(); this.metricsCollection = new ArrayList<>(); this.tempMetricsCollection = new ArrayList<>(); // schedule stat generation task every 1 minute From 216b83008deb469e0fc55ed8117f0c393ebcb0ac Mon Sep 17 00:00:00 2001 From: maheshnikam <55378196+nikam14@users.noreply.github.com> Date: Mon, 23 Sep 2024 14:17:31 +0530 Subject: [PATCH 002/327] [fix][test] Fix flaky test LeaderElectionTest.revalidateLeaderWithinSameSession (#22383) --- .../java/org/apache/pulsar/metadata/api/MetadataCache.java | 2 +- .../org/apache/pulsar/metadata/BaseMetadataStoreTest.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java index 8e153b23d3087..4af712d33571e 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java @@ -59,7 +59,7 @@ public interface MetadataCache { * * @param path * the path of the object in the metadata store - * @return the cached object or an empty {@link Optional} is the cache doesn't have the object + * @return the cached object or an empty {@link Optional} is the cache does not have the object */ Optional getIfCached(String path); 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 c77de92ae3c4c..d0265e3ca44ee 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 @@ -145,10 +145,11 @@ public static void assertEqualsAndRetry(Supplier actual, int retryCount, long intSleepTimeInMillis) throws Exception { assertTrue(retryStrategically((__) -> { - if (actual.get().equals(expectedAndRetry)) { + Object actualObject = actual.get(); + if (actualObject.equals(expectedAndRetry)) { return false; } - assertEquals(actual.get(), expected); + assertEquals(actualObject, expected); return true; }, retryCount, intSleepTimeInMillis)); } From 7d4ac9dc542ff1e840f4e520836b6a3c49c6338d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 23 Sep 2024 15:00:52 +0300 Subject: [PATCH 003/327] [fix][build] Fix problem where git.commit.id.abbrev is missing in image tagging (#23337) --- docker/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/pom.xml b/docker/pom.xml index a5ea238241c6a..ffcaec3ffdc30 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -68,7 +68,6 @@ false true - true false From 501dfdeace9ef321acbdc5ce32d98eb3e56e083a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 23 Sep 2024 16:37:33 +0300 Subject: [PATCH 004/327] [fix][sec] Upgrade vertx to 4.5.10 to address CVE-2024-8391 (#23338) --- distribution/server/src/assemble/LICENSE.bin.txt | 10 +++++----- pom.xml | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 579613b0d8f2f..1d78913849bda 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -491,11 +491,11 @@ The Apache Software License, Version 2.0 * JCTools - Java Concurrency Tools for the JVM - org.jctools-jctools-core-2.1.2.jar * Vertx - - io.vertx-vertx-auth-common-4.5.8.jar - - io.vertx-vertx-bridge-common-4.5.8.jar - - io.vertx-vertx-core-4.5.8.jar - - io.vertx-vertx-web-4.5.8.jar - - io.vertx-vertx-web-common-4.5.8.jar + - io.vertx-vertx-auth-common-4.5.10.jar + - io.vertx-vertx-bridge-common-4.5.10.jar + - io.vertx-vertx-core-4.5.10.jar + - io.vertx-vertx-web-4.5.10.jar + - io.vertx-vertx-web-common-4.5.10.jar * Apache ZooKeeper - org.apache.zookeeper-zookeeper-3.9.2.jar - org.apache.zookeeper-zookeeper-jute-3.9.2.jar diff --git a/pom.xml b/pom.xml index dda7f316acc6f..1f7ecd3b90c1e 100644 --- a/pom.xml +++ b/pom.xml @@ -155,7 +155,7 @@ flexible messaging model and an intuitive client API. 2.42 1.10.50 0.16.0 - 4.5.8 + 4.5.10 7.9.2 2.0.13 4.4 From 4ce0c752cc4b2d6dccb818ab0ffa854e82e42b85 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 24 Sep 2024 03:50:01 +0800 Subject: [PATCH 005/327] [fix] Key_Shared mode consumption latency when low traffic (#23340) Co-authored-by: Lari Hotari --- conf/broker.conf | 4 ++-- conf/standalone.conf | 4 ++-- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 4 ++-- .../persistent/PersistentDispatcherMultipleConsumers.java | 6 ++++-- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 125b2aa8c1b39..617e202e5ec65 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -489,12 +489,12 @@ dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 # On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered # out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff # delay. This parameter sets the initial backoff delay in milliseconds. -dispatcherRetryBackoffInitialTimeInMs=100 +dispatcherRetryBackoffInitialTimeInMs=1 # On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered # out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff # delay. This parameter sets the maximum backoff delay in milliseconds. -dispatcherRetryBackoffMaxTimeInMs=1000 +dispatcherRetryBackoffMaxTimeInMs=10 # Precise dispatcher flow control according to history message number of each entry preciseDispatcherFlowControl=false diff --git a/conf/standalone.conf b/conf/standalone.conf index 622949bf6c325..535800a43f3e0 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -305,12 +305,12 @@ dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 # On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered # out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff # delay. This parameter sets the initial backoff delay in milliseconds. -dispatcherRetryBackoffInitialTimeInMs=100 +dispatcherRetryBackoffInitialTimeInMs=1 # On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered # out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff # delay. This parameter sets the maximum backoff delay in milliseconds. -dispatcherRetryBackoffMaxTimeInMs=1000 +dispatcherRetryBackoffMaxTimeInMs=10 # Precise dispatcher flow control according to history message number of each entry preciseDispatcherFlowControl=false diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 486587ec174a0..33b4fbff5f5bb 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -1231,14 +1231,14 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered " + "out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff " + "delay. This parameter sets the initial backoff delay in milliseconds.") - private int dispatcherRetryBackoffInitialTimeInMs = 100; + private int dispatcherRetryBackoffInitialTimeInMs = 1; @FieldContext( category = CATEGORY_POLICIES, doc = "On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered " + "out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff " + "delay. This parameter sets the maximum backoff delay in milliseconds.") - private int dispatcherRetryBackoffMaxTimeInMs = 1000; + private int dispatcherRetryBackoffMaxTimeInMs = 10; @FieldContext( dynamic = true, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 450a446c85a78..8fdb65e7b3076 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -729,11 +729,13 @@ private synchronized void handleSendingMessagesAndReadingMore(ReadType readType, boolean triggerReadingMore = sendMessagesToConsumers(readType, entries, needAcquireSendInProgress); int entriesDispatched = lastNumberOfEntriesDispatched; updatePendingBytesToDispatch(-totalBytesSize); + if (entriesDispatched > 0) { + // Reset the backoff when we successfully dispatched messages + retryBackoff.reset(); + } if (triggerReadingMore) { if (entriesDispatched > 0 || skipNextBackoff) { skipNextBackoff = false; - // Reset the backoff when we successfully dispatched messages - retryBackoff.reset(); // Call readMoreEntries in the same thread to trigger the next read readMoreEntries(); } else if (entriesDispatched == 0) { From 5ea4252a493c5d93046cfc3aeb1977814bc64a41 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 25 Sep 2024 08:28:21 +0300 Subject: [PATCH 006/327] [fix][broker] Avoid introducing delay when there are delayed messages or marker messages (#23343) --- ...PersistentDispatcherMultipleConsumers.java | 34 +++++--- ...tStickyKeyDispatcherMultipleConsumers.java | 11 ++- ...ckyKeyDispatcherMultipleConsumersTest.java | 82 +++++++++++++++++++ 3 files changed, 112 insertions(+), 15 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 8fdb65e7b3076..73ad2cf0a3dee 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -134,7 +134,11 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul private AtomicBoolean isRescheduleReadInProgress = new AtomicBoolean(false); protected final ExecutorService dispatchMessagesThread; private final SharedConsumerAssignor assignor; - protected int lastNumberOfEntriesDispatched; + // tracks how many entries were processed by consumers in the last trySendMessagesToConsumers call + // the number includes also delayed messages, marker messages, aborted txn messages and filtered messages + // When no messages were processed, the value is 0. This is also an indication that the dispatcher didn't + // make progress in the last trySendMessagesToConsumers call. + protected int lastNumberOfEntriesProcessed; protected boolean skipNextBackoff; private final Backoff retryBackoff; protected enum ReadType { @@ -727,19 +731,22 @@ private synchronized void handleSendingMessagesAndReadingMore(ReadType readType, boolean needAcquireSendInProgress, long totalBytesSize) { boolean triggerReadingMore = sendMessagesToConsumers(readType, entries, needAcquireSendInProgress); - int entriesDispatched = lastNumberOfEntriesDispatched; + int entriesProcessed = lastNumberOfEntriesProcessed; updatePendingBytesToDispatch(-totalBytesSize); - if (entriesDispatched > 0) { - // Reset the backoff when we successfully dispatched messages + boolean canReadMoreImmediately = false; + if (entriesProcessed > 0 || skipNextBackoff) { + // Reset the backoff when messages were processed retryBackoff.reset(); + // Reset the possible flag to skip the backoff delay + skipNextBackoff = false; + canReadMoreImmediately = true; } if (triggerReadingMore) { - if (entriesDispatched > 0 || skipNextBackoff) { - skipNextBackoff = false; + if (canReadMoreImmediately) { // Call readMoreEntries in the same thread to trigger the next read readMoreEntries(); - } else if (entriesDispatched == 0) { - // If no messages were dispatched, we need to reschedule a new read with an increasing backoff delay + } else { + // reschedule a new read with an increasing backoff delay reScheduleReadWithBackoff(); } } @@ -779,7 +786,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis if (needTrimAckedMessages()) { cursor.trimDeletedEntries(entries); } - lastNumberOfEntriesDispatched = 0; + lastNumberOfEntriesProcessed = 0; int entriesToDispatch = entries.size(); // Trigger read more messages @@ -809,6 +816,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis long totalMessagesSent = 0; long totalBytesSent = 0; long totalEntries = 0; + long totalEntriesProcessed = 0; int avgBatchSizePerMsg = remainingMessages > 0 ? Math.max(remainingMessages / entries.size(), 1) : 1; // If the dispatcher is closed, firstAvailableConsumerPermits will be 0, which skips dispatching the @@ -820,6 +828,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis log.info("[{}] rewind because no available consumer found from total {}", name, consumerList.size()); entries.subList(start, entries.size()).forEach(Entry::release); cursor.rewind(); + lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; return false; } @@ -863,6 +872,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis totalEntries += filterEntriesForConsumer(metadataArray, start, entriesForThisConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay, c); + totalEntriesProcessed += entriesForThisConsumer.size(); c.sendMessages(entriesForThisConsumer, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), redeliveryTracker); @@ -882,7 +892,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis totalBytesSent += sendMessageInfo.getTotalBytes(); } - lastNumberOfEntriesDispatched = (int) totalEntries; + lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); if (entriesToDispatch > 0) { @@ -917,6 +927,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, long totalMessagesSent = 0; long totalBytesSent = 0; long totalEntries = 0; + long totalEntriesProcessed = 0; final AtomicInteger numConsumers = new AtomicInteger(assignResult.size()); for (Map.Entry> current : assignResult.entrySet()) { final Consumer consumer = current.getKey(); @@ -947,6 +958,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, totalEntries += filterEntriesForConsumer(entryAndMetadataList, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + totalEntriesProcessed += entryAndMetadataList.size(); consumer.sendMessages(entryAndMetadataList, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker() @@ -962,7 +974,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, totalBytesSent += sendMessageInfo.getTotalBytes(); } - lastNumberOfEntriesDispatched = (int) totalEntries; + lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); return numConsumers.get() == 0; // trigger a new readMoreEntries() call diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 26463ba902c58..ecd3f19a14028 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -190,10 +190,11 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE @Override protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { - lastNumberOfEntriesDispatched = 0; + lastNumberOfEntriesProcessed = 0; long totalMessagesSent = 0; long totalBytesSent = 0; long totalEntries = 0; + long totalEntriesProcessed = 0; int entriesCount = entries.size(); // Trigger read more messages @@ -233,6 +234,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } else if (readType == ReadType.Replay) { entries.forEach(Entry::release); } + skipNextBackoff = true; return true; } } @@ -298,6 +300,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(entriesForConsumer.size()); totalEntries += filterEntriesForConsumer(entriesForConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + totalEntriesProcessed += entriesForConsumer.size(); consumer.sendMessages(entriesForConsumer, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), @@ -368,7 +371,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } } - lastNumberOfEntriesDispatched = (int) totalEntries; + lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; // acquire message-dispatch permits for already delivered messages acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); @@ -387,8 +390,8 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis return true; } - // if no messages were sent, we should retry after a backoff delay - if (entriesByConsumerForDispatching.size() == 0) { + // if no messages were sent to consumers, we should retry + if (totalEntries == 0) { return true; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index b78d1e554c32d..dcd852f409dbb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -46,6 +46,8 @@ import io.netty.buffer.Unpooled; import io.netty.channel.ChannelPromise; import io.netty.channel.EventLoopGroup; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; @@ -996,6 +998,86 @@ protected void reScheduleReadInMs(long readAfterMs) { ); } + + @Test(dataProvider = "testBackoffDelayWhenNoMessagesDispatched") + public void testNoBackoffDelayWhenDelayedMessages(boolean dispatchMessagesInSubscriptionThread, boolean isKeyShared) + throws Exception { + persistentDispatcher.close(); + + doReturn(dispatchMessagesInSubscriptionThread).when(configMock) + .isDispatcherDispatchMessagesInSubscriptionThread(); + + AtomicInteger readMoreEntriesCalled = new AtomicInteger(0); + AtomicInteger reScheduleReadInMsCalled = new AtomicInteger(0); + AtomicBoolean delayAllMessages = new AtomicBoolean(true); + + PersistentDispatcherMultipleConsumers dispatcher; + if (isKeyShared) { + dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers( + topicMock, cursorMock, subscriptionMock, configMock, + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)) { + @Override + protected void reScheduleReadInMs(long readAfterMs) { + reScheduleReadInMsCalled.incrementAndGet(); + } + + @Override + public synchronized void readMoreEntries() { + readMoreEntriesCalled.incrementAndGet(); + } + + @Override + public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata msgMetadata) { + if (delayAllMessages.get()) { + // simulate delayed message + return true; + } + return super.trackDelayedDelivery(ledgerId, entryId, msgMetadata); + } + }; + } else { + dispatcher = new PersistentDispatcherMultipleConsumers(topicMock, cursorMock, subscriptionMock) { + @Override + protected void reScheduleReadInMs(long readAfterMs) { + reScheduleReadInMsCalled.incrementAndGet(); + } + + @Override + public synchronized void readMoreEntries() { + readMoreEntriesCalled.incrementAndGet(); + } + + @Override + public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata msgMetadata) { + if (delayAllMessages.get()) { + // simulate delayed message + return true; + } + return super.trackDelayedDelivery(ledgerId, entryId, msgMetadata); + } + }; + } + + doAnswer(invocationOnMock -> { + GenericFutureListener> listener = invocationOnMock.getArgument(0); + Future future = mock(Future.class); + when(future.isDone()).thenReturn(true); + listener.operationComplete(future); + return channelMock; + }).when(channelMock).addListener(any()); + + // add a consumer with permits + consumerMockAvailablePermits.set(1000); + dispatcher.addConsumer(consumerMock); + + List entries = new ArrayList<>(List.of(EntryImpl.create(1, 1, createMessage("message1", 1)))); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + Awaitility.await().untilAsserted(() -> { + assertEquals(reScheduleReadInMsCalled.get(), 0, "reScheduleReadInMs should not be called"); + assertTrue(readMoreEntriesCalled.get() >= 1); + }); + } + private ByteBuf createMessage(String message, int sequenceId) { return createMessage(message, sequenceId, "testKey"); } From cefa72c7d18ab05d413e8737672f746629874f59 Mon Sep 17 00:00:00 2001 From: maheshnikam <55378196+nikam14@users.noreply.github.com> Date: Wed, 25 Sep 2024 20:15:12 +0530 Subject: [PATCH 007/327] [improve][schema] Improve Incompatible Schema Exception error message in ServerCnx.java (#23344) --- .../java/org/apache/pulsar/broker/service/ServerCnx.java | 5 +++-- 1 file changed, 3 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 7d196ad30235b..5b67b01115e7c 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 @@ -3052,7 +3052,8 @@ private CompletableFuture tryAddSchema(Topic topic, SchemaData sc CompletableFuture result = new CompletableFuture<>(); if (hasSchema && (schemaValidationEnforced || topic.getSchemaValidationEnforced())) { result.completeExceptionally(new IncompatibleSchemaException( - "Producers cannot connect or send message without a schema to topics with a schema")); + "Producers cannot connect or send message without a schema to topics with a schema" + + "when SchemaValidationEnforced is enabled")); } else { result.complete(SchemaVersion.Empty); } @@ -3727,4 +3728,4 @@ public void incrementThrottleCount() { public void decrementThrottleCount() { throttleTracker.decrementThrottleCount(); } -} \ No newline at end of file +} From 31f27a091920dfcdd9ae44a1c738d701294b318b Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 25 Sep 2024 14:01:08 -0700 Subject: [PATCH 008/327] [fix][cli] Fix Pulsar-Client to allow consume encrypted messages with appropriate crypto-failure-action (#23346) --- .../main/java/org/apache/pulsar/client/cli/CmdConsume.java | 5 +++++ .../src/main/java/org/apache/pulsar/client/cli/CmdRead.java | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java index 71c172b633713..46adc45e2ea4d 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; @@ -111,6 +112,9 @@ public class CmdConsume extends AbstractCmdConsume { @Option(names = {"-rs", "--replicated" }, description = "Whether the subscription status should be replicated") private boolean replicateSubscriptionState = false; + @Option(names = { "-ca", "--crypto-failure-action" }, description = "Crypto Failure Action") + private ConsumerCryptoFailureAction cryptoFailureAction = ConsumerCryptoFailureAction.FAIL; + public CmdConsume() { // Do nothing super(); @@ -174,6 +178,7 @@ private int consume(String topic) { } builder.autoAckOldestChunkedMessageOnQueueFull(this.autoAckOldestChunkedMessageOnQueueFull); + builder.cryptoFailureAction(cryptoFailureAction); if (isNotBlank(this.encKeyValue)) { builder.defaultCryptoKeyReader(this.encKeyValue); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java index daab436499219..51bf2d6898b6b 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java @@ -30,6 +30,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; @@ -101,6 +102,9 @@ public class CmdRead extends AbstractCmdConsume { @Option(names = { "-pm", "--pool-messages" }, description = "Use the pooled message", arity = "1") private boolean poolMessages = true; + @Option(names = { "-ca", "--crypto-failure-action" }, description = "Crypto Failure Action") + private ConsumerCryptoFailureAction cryptoFailureAction = ConsumerCryptoFailureAction.FAIL; + public CmdRead() { // Do nothing super(); @@ -153,6 +157,7 @@ private int read(String topic) { } builder.autoAckOldestChunkedMessageOnQueueFull(this.autoAckOldestChunkedMessageOnQueueFull); + builder.cryptoFailureAction(cryptoFailureAction); if (isNotBlank(this.encKeyValue)) { builder.defaultCryptoKeyReader(this.encKeyValue); From b1c5d96ab480ba1b8ad8cbe2077cbe4c467dfc61 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 25 Sep 2024 15:30:55 -0700 Subject: [PATCH 009/327] [fix][cli] Fix Pulsar-Client CLI to print metadata of message including encryption metadata (#23347) --- .../pulsar/client/cli/AbstractCmdConsume.java | 45 ++++++++++++++++++- .../apache/pulsar/client/cli/CmdConsume.java | 5 ++- .../org/apache/pulsar/client/cli/CmdRead.java | 5 ++- 3 files changed, 52 insertions(+), 3 deletions(-) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java index 658b34767b594..33df4aca96d2d 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Base64; import java.util.HashMap; import java.util.Map; import java.util.concurrent.BlockingQueue; @@ -40,7 +41,9 @@ import org.apache.pulsar.client.api.schema.Field; import org.apache.pulsar.client.api.schema.GenericObject; import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.common.api.EncryptionContext; import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue; import org.eclipse.jetty.websocket.api.RemoteEndpoint; import org.eclipse.jetty.websocket.api.Session; @@ -87,7 +90,8 @@ public void updateConfig(ClientBuilder clientBuilder, Authentication authenticat * Whether to display BytesMessages in hexdump style, ignored for simple text messages * @return String representation of the message */ - protected String interpretMessage(Message message, boolean displayHex) throws IOException { + protected String interpretMessage(Message message, boolean displayHex, boolean printMetadata) + throws IOException { StringBuilder sb = new StringBuilder(); String properties = Arrays.toString(message.getProperties().entrySet().toArray()); @@ -122,6 +126,45 @@ protected String interpretMessage(Message message, boolean displayHex) throws } sb.append("content:").append(data); + if (printMetadata) { + if (message.getEncryptionCtx().isPresent()) { + EncryptionContext encContext = message.getEncryptionCtx().get(); + if (encContext.getKeys() != null && !encContext.getKeys().isEmpty()) { + sb.append(", "); + sb.append("encryption-keys:").append(", "); + encContext.getKeys().forEach((keyName, keyInfo) -> { + String metadata = Arrays.toString(keyInfo.getMetadata().entrySet().toArray()); + sb.append("name:").append(keyName).append(", ").append("key-value:") + .append(Base64.getEncoder().encode(keyInfo.getKeyValue())).append(", ") + .append("metadata:").append(metadata).append(", "); + + }); + sb.append(", ").append("param:").append(Base64.getEncoder().encode(encContext.getParam())) + .append(", ").append("algorithm:").append(encContext.getAlgorithm()).append(", ") + .append("compression-type:").append(encContext.getCompressionType()).append(", ") + .append("uncompressed-size").append(encContext.getUncompressedMessageSize()).append(", ") + .append("batch-size") + .append(encContext.getBatchSize().isPresent() ? encContext.getBatchSize().get() : 1); + } + } + if (message.hasBrokerPublishTime()) { + sb.append(", ").append("publish-time:").append(DateFormatter.format(message.getPublishTime())); + } + sb.append(", ").append("event-time:").append(DateFormatter.format(message.getEventTime())); + sb.append(", ").append("message-id:").append(message.getMessageId()); + sb.append(", ").append("producer-name:").append(message.getProducerName()); + sb.append(", ").append("sequence-id:").append(message.getSequenceId()); + sb.append(", ").append("replicated-from:").append(message.getReplicatedFrom()); + sb.append(", ").append("redelivery-count:").append(message.getRedeliveryCount()); + sb.append(", ").append("ordering-key:") + .append(message.getOrderingKey() != null ? new String(message.getOrderingKey()) : ""); + sb.append(", ").append("schema-version:") + .append(message.getSchemaVersion() != null ? new String(message.getSchemaVersion()) : ""); + if (message.hasIndex()) { + sb.append(", ").append("index:").append(message.getIndex()); + } + } + return sb.toString(); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java index 46adc45e2ea4d..0f0e2f0a9c813 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java @@ -115,6 +115,9 @@ public class CmdConsume extends AbstractCmdConsume { @Option(names = { "-ca", "--crypto-failure-action" }, description = "Crypto Failure Action") private ConsumerCryptoFailureAction cryptoFailureAction = ConsumerCryptoFailureAction.FAIL; + @Option(names = { "-mp", "--print-metadata" }, description = "Message metadata") + private boolean printMetadata = false; + public CmdConsume() { // Do nothing super(); @@ -199,7 +202,7 @@ private int consume(String topic) { numMessagesConsumed += 1; if (!hideContent) { System.out.println(MESSAGE_BOUNDARY); - String output = this.interpretMessage(msg, displayHex); + String output = this.interpretMessage(msg, displayHex, printMetadata); System.out.println(output); } else if (numMessagesConsumed % 1000 == 0) { System.out.println("Received " + numMessagesConsumed + " messages"); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java index 51bf2d6898b6b..529d1d9c41272 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java @@ -105,6 +105,9 @@ public class CmdRead extends AbstractCmdConsume { @Option(names = { "-ca", "--crypto-failure-action" }, description = "Crypto Failure Action") private ConsumerCryptoFailureAction cryptoFailureAction = ConsumerCryptoFailureAction.FAIL; + @Option(names = { "-mp", "--print-metadata" }, description = "Message metadata") + private boolean printMetadata = false; + public CmdRead() { // Do nothing super(); @@ -178,7 +181,7 @@ private int read(String topic) { numMessagesRead += 1; if (!hideContent) { System.out.println(MESSAGE_BOUNDARY); - String output = this.interpretMessage(msg, displayHex); + String output = this.interpretMessage(msg, displayHex, printMetadata); System.out.println(output); } else if (numMessagesRead % 1000 == 0) { System.out.println("Received " + numMessagesRead + " messages"); From 5583102aae135f5f62884f83e1ddd927b24ee737 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 27 Sep 2024 16:34:19 +0800 Subject: [PATCH 010/327] [fix] [log] Do not print error log if tenant/namespace does not exist when calling get topic metadata (#23291) --- .../pulsar/broker/service/ServerCnx.java | 41 ++++++++++++--- .../admin/GetPartitionMetadataTest.java | 51 +++++++++++++++++++ 2 files changed, 85 insertions(+), 7 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 5b67b01115e7c..aedd68d416fe7 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 @@ -63,6 +63,8 @@ import java.util.stream.Collectors; import javax.naming.AuthenticationException; import javax.net.ssl.SSLSession; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Response; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedger; @@ -672,8 +674,6 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa commandSender.sendPartitionMetadataResponse(ServerError.AuthorizationError, ex.getMessage(), requestId); } else { - log.warn("Failed to get Partitioned Metadata [{}] {}: {}", remoteAddress, - topicName, ex.getMessage(), ex); ServerError error = ServerError.ServiceNotReady; if (ex instanceof MetadataStoreException) { error = ServerError.MetadataError; @@ -685,6 +685,14 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa error = ServerError.MetadataError; } } + if (error == ServerError.TopicNotFound) { + log.info("Trying to get Partitioned Metadata for a resource not exist" + + "[{}] {}: {}", remoteAddress, + topicName, ex.getMessage()); + } else { + log.warn("Failed to get Partitioned Metadata [{}] {}: {}", + remoteAddress, topicName, ex.getMessage(), ex); + } commandSender.sendPartitionMetadataResponse(error, ex.getMessage(), requestId); } @@ -702,6 +710,16 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa return null; }).exceptionally(ex -> { logAuthException(remoteAddress, "partition-metadata", getPrincipal(), Optional.of(topicName), ex); + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (actEx instanceof WebApplicationException restException) { + if (restException.getResponse().getStatus() == Response.Status.NOT_FOUND.getStatusCode()) { + writeAndFlush(Commands.newPartitionMetadataResponse(ServerError.MetadataError, + "Tenant or namespace or topic does not exist: " + topicName.getNamespace() , + requestId)); + lookupSemaphore.release(); + return null; + } + } final String msg = "Exception occurred while trying to authorize get Partition Metadata"; writeAndFlush(Commands.newPartitionMetadataResponse(ServerError.AuthorizationError, msg, requestId)); @@ -3663,13 +3681,22 @@ protected void messageReceived() { private static void logAuthException(SocketAddress remoteAddress, String operation, String principal, Optional topic, Throwable ex) { String topicString = topic.map(t -> ", topic=" + t.toString()).orElse(""); - if (ex instanceof AuthenticationException) { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (actEx instanceof AuthenticationException) { log.info("[{}] Failed to authenticate: operation={}, principal={}{}, reason={}", - remoteAddress, operation, principal, topicString, ex.getMessage()); - } else { - log.error("[{}] Error trying to authenticate: operation={}, principal={}{}", - remoteAddress, operation, principal, topicString, ex); + remoteAddress, operation, principal, topicString, actEx.getMessage()); + return; + } else if (actEx instanceof WebApplicationException restException){ + // Do not print error log if users tries to access a not found resource. + if (restException.getResponse().getStatus() == Response.Status.NOT_FOUND.getStatusCode()) { + log.info("[{}] Trying to authenticate for a topic which under a namespace not exists: operation={}," + + " principal={}{}, reason: {}", + remoteAddress, operation, principal, topicString, actEx.getMessage()); + return; + } } + log.error("[{}] Error trying to authenticate: operation={}, principal={}{}", + remoteAddress, operation, principal, topicString, ex); } private static void logNamespaceNameAuthException(SocketAddress remoteAddress, String operation, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java index 87bc4267b48a3..e9a639697d9ff 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java @@ -578,4 +578,55 @@ public void testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(boolean config assertEquals(getLookupRequestPermits(), lookupPermitsBefore); }); } + + @Test(dataProvider = "topicDomains") + public void testNamespaceNotExist(TopicDomain topicDomain) throws Exception { + int lookupPermitsBefore = getLookupRequestPermits(); + final String namespaceNotExist = BrokerTestUtil.newUniqueName("public/ns"); + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.toString() + "://" + namespaceNotExist + "/tp"); + PulsarClientImpl[] clientArray = getClientsToTest(false); + for (PulsarClientImpl client : clientArray) { + try { + PartitionedTopicMetadata topicMetadata = client + .getPartitionedTopicMetadata(topicNameStr, true, true) + .join(); + log.info("Get topic metadata: {}", topicMetadata.partitions); + fail("Expected a not found ex"); + } catch (Exception ex) { + Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); + assertTrue(unwrapEx instanceof PulsarClientException.BrokerMetadataException || + unwrapEx instanceof PulsarClientException.TopicDoesNotExistException); + } + } + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } + + @Test(dataProvider = "topicDomains") + public void testTenantNotExist(TopicDomain topicDomain) throws Exception { + int lookupPermitsBefore = getLookupRequestPermits(); + final String tenantNotExist = BrokerTestUtil.newUniqueName("tenant"); + final String namespaceNotExist = BrokerTestUtil.newUniqueName(tenantNotExist + "/default"); + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.toString() + "://" + namespaceNotExist + "/tp"); + PulsarClientImpl[] clientArray = getClientsToTest(false); + for (PulsarClientImpl client : clientArray) { + try { + PartitionedTopicMetadata topicMetadata = client + .getPartitionedTopicMetadata(topicNameStr, true, true) + .join(); + log.info("Get topic metadata: {}", topicMetadata.partitions); + fail("Expected a not found ex"); + } catch (Exception ex) { + Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); + assertTrue(unwrapEx instanceof PulsarClientException.BrokerMetadataException || + unwrapEx instanceof PulsarClientException.TopicDoesNotExistException); + } + } + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } } From e91574ac7b44348a05f1ae812c5aae3abb26fe64 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 27 Sep 2024 17:33:28 +0800 Subject: [PATCH 011/327] [fix][broker] Fix unloadNamespaceBundlesGracefully can be stuck with extensible load manager (#23349) --- .../apache/pulsar/broker/PulsarService.java | 7 +- .../extensions/ExtensibleLoadManagerImpl.java | 130 +++++++++++++----- .../ExtensibleLoadManagerWrapper.java | 2 +- .../channel/ServiceUnitStateChannelImpl.java | 43 ++++-- .../channel/ServiceUnitStateData.java | 2 +- .../ServiceUnitStateDataConflictResolver.java | 2 +- .../ServiceUnitStateTableViewImpl.java | 19 ++- .../filter/BrokerMaxTopicCountFilter.java | 7 +- .../store/TableViewLoadDataStoreImpl.java | 10 +- .../pulsar/broker/service/BrokerService.java | 7 +- .../SystemTopicBasedTopicPoliciesService.java | 32 +++-- .../ExtensibleLoadManagerCloseTest.java | 50 +++++-- .../extensions/store/LoadDataStoreTest.java | 3 +- .../pulsar/client/impl/TableViewImpl.java | 7 +- 14 files changed, 240 insertions(+), 81 deletions(-) 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 a2f6fb9e9773b..6c768a078974f 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 @@ -513,6 +513,9 @@ public CompletableFuture closeAsync() { return closeFuture; } LOG.info("Closing PulsarService"); + if (topicPoliciesService != null) { + topicPoliciesService.close(); + } if (brokerService != null) { brokerService.unloadNamespaceBundlesGracefully(); } @@ -633,10 +636,6 @@ public CompletableFuture closeAsync() { transactionBufferClient.close(); } - if (topicPoliciesService != null) { - topicPoliciesService.close(); - topicPoliciesService = null; - } if (client != null) { client.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 98ef6bf36edac..841f9bfb669d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -181,7 +181,14 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private SplitManager splitManager; - volatile boolean started = false; + enum State { + INIT, + RUNNING, + // It's removing visibility of the current broker from other brokers. In this state, it cannot play as a leader + // or follower. + DISABLED, + } + private final AtomicReference state = new AtomicReference<>(State.INIT); private boolean configuredSystemTopics = false; @@ -214,7 +221,7 @@ public CompletableFuture> getOwnedServiceUnitsAsync() { } public Set getOwnedServiceUnits() { - if (!started) { + if (state.get() == State.INIT) { log.warn("Failed to get owned service units, load manager is not started."); return Collections.emptySet(); } @@ -344,7 +351,7 @@ public static CompletableFuture> getAssignedBrokerLoo @Override public void start() throws PulsarServerException { - if (this.started) { + if (state.get() != State.INIT) { return; } try { @@ -443,7 +450,9 @@ public void start() throws PulsarServerException { this.splitScheduler.start(); this.initWaiter.complete(true); - this.started = true; + if (!state.compareAndSet(State.INIT, State.RUNNING)) { + failForUnexpectedState("start"); + } log.info("Started load manager."); } catch (Throwable e) { failStarting(e); @@ -615,21 +624,17 @@ public CompletableFuture> selectAsync(ServiceUnitId bundle, filter.filterAsync(availableBrokerCandidates, bundle, context); futures.add(future); } - CompletableFuture> result = new CompletableFuture<>(); - FutureUtil.waitForAll(futures).whenComplete((__, ex) -> { - if (ex != null) { - // TODO: We may need to revisit this error case. - log.error("Failed to filter out brokers when select bundle: {}", bundle, ex); - } + return FutureUtil.waitForAll(futures).exceptionally(e -> { + // TODO: We may need to revisit this error case. + log.error("Failed to filter out brokers when select bundle: {}", bundle, e); + return null; + }).thenApply(__ -> { if (availableBrokerCandidates.isEmpty()) { - result.complete(Optional.empty()); - return; + return Optional.empty(); } Set candidateBrokers = availableBrokerCandidates.keySet(); - - result.complete(getBrokerSelectionStrategy().select(candidateBrokers, bundle, context)); + return getBrokerSelectionStrategy().select(candidateBrokers, bundle, context); }); - return result; }); } @@ -667,6 +672,9 @@ public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle, boolean force, long timeout, TimeUnit timeoutUnit) { + if (state.get() == State.INIT) { + return CompletableFuture.completedFuture(null); + } if (NamespaceService.isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) { log.info("Skip unloading namespace bundle: {}.", bundle); return CompletableFuture.completedFuture(null); @@ -755,24 +763,11 @@ private CompletableFuture splitAsync(SplitDecision decision, @Override public void close() throws PulsarServerException { - if (!this.started) { + if (state.get() == State.INIT) { return; } try { - if (brokerLoadDataReportTask != null) { - brokerLoadDataReportTask.cancel(true); - } - - if (topBundlesLoadDataReportTask != null) { - topBundlesLoadDataReportTask.cancel(true); - } - - if (monitorTask != null) { - monitorTask.cancel(true); - } - - this.brokerLoadDataStore.shutdown(); - this.topBundlesLoadDataStore.shutdown(); + stopLoadDataReportTasks(); this.unloadScheduler.close(); this.splitScheduler.close(); this.serviceUnitStateTableViewSyncer.close(); @@ -791,7 +786,7 @@ public void close() throws PulsarServerException { } catch (Exception e) { throw new PulsarServerException(e); } finally { - this.started = false; + state.set(State.INIT); } } @@ -799,6 +794,28 @@ public void close() throws PulsarServerException { } } + private void stopLoadDataReportTasks() { + if (brokerLoadDataReportTask != null) { + brokerLoadDataReportTask.cancel(true); + } + if (topBundlesLoadDataReportTask != null) { + topBundlesLoadDataReportTask.cancel(true); + } + if (monitorTask != null) { + monitorTask.cancel(true); + } + try { + brokerLoadDataStore.shutdown(); + } catch (IOException e) { + log.warn("Failed to shutdown brokerLoadDataStore", e); + } + try { + topBundlesLoadDataStore.shutdown(); + } catch (IOException e) { + log.warn("Failed to shutdown topBundlesLoadDataStore", e); + } + } + public static boolean isInternalTopic(String topic) { return INTERNAL_TOPICS.contains(topic) || topic.startsWith(TOPIC) @@ -814,13 +831,16 @@ synchronized void playLeader() { boolean becameFollower = false; while (!Thread.currentThread().isInterrupted()) { try { - if (!initWaiter.get()) { + if (!initWaiter.get() || disabled()) { return; } if (!serviceUnitStateChannel.isChannelOwner()) { becameFollower = true; break; } + if (disabled()) { + return; + } // Confirm the system topics have been created or create them if they do not exist. // If the leader has changed, the new leader need to reset // the local brokerService.topics (by this topic creations). @@ -835,6 +855,11 @@ synchronized void playLeader() { } break; } catch (Throwable e) { + if (disabled()) { + log.warn("The broker:{} failed to set the role but exit because it's disabled", + pulsar.getBrokerId(), e); + return; + } log.warn("The broker:{} failed to set the role. Retrying {} th ...", pulsar.getBrokerId(), ++retry, e); try { @@ -846,6 +871,9 @@ synchronized void playLeader() { } } } + if (disabled()) { + return; + } if (becameFollower) { log.warn("The broker:{} became follower while initializing leader role.", pulsar.getBrokerId()); @@ -869,13 +897,16 @@ synchronized void playFollower() { boolean becameLeader = false; while (!Thread.currentThread().isInterrupted()) { try { - if (!initWaiter.get()) { + if (!initWaiter.get() || disabled()) { return; } if (serviceUnitStateChannel.isChannelOwner()) { becameLeader = true; break; } + if (disabled()) { + return; + } unloadScheduler.close(); serviceUnitStateChannel.cancelOwnershipMonitor(); closeInternalTopics(); @@ -885,6 +916,11 @@ synchronized void playFollower() { serviceUnitStateTableViewSyncer.close(); break; } catch (Throwable e) { + if (disabled()) { + log.warn("The broker:{} failed to set the role but exit because it's disabled", + pulsar.getBrokerId(), e); + return; + } log.warn("The broker:{} failed to set the role. Retrying {} th ...", pulsar.getBrokerId(), ++retry, e); try { @@ -896,6 +932,9 @@ synchronized void playFollower() { } } } + if (disabled()) { + return; + } if (becameLeader) { log.warn("This broker:{} became leader while initializing follower role.", pulsar.getBrokerId()); @@ -982,9 +1021,20 @@ protected void monitor() { } public void disableBroker() throws Exception { + // TopicDoesNotExistException might be thrown and it's not recoverable. Enable this flag to exit playFollower() + // or playLeader() quickly. + if (!state.compareAndSet(State.RUNNING, State.DISABLED)) { + failForUnexpectedState("disableBroker"); + } + stopLoadDataReportTasks(); serviceUnitStateChannel.cleanOwnerships(); - leaderElectionService.close(); brokerRegistry.unregister(); + leaderElectionService.close(); + final var availableBrokers = brokerRegistry.getAvailableBrokersAsync() + .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); + if (availableBrokers.isEmpty()) { + close(); + } // Close the internal topics (if owned any) after giving up the possible leader role, // so that the subsequent lookups could hit the next leader. closeInternalTopics(); @@ -1018,4 +1068,16 @@ protected BrokerRegistry createBrokerRegistry(PulsarService pulsar) { protected ServiceUnitStateChannel createServiceUnitStateChannel(PulsarService pulsar) { return new ServiceUnitStateChannelImpl(pulsar); } + + private void failForUnexpectedState(String msg) { + throw new IllegalStateException("Failed to " + msg + ", state: " + state.get()); + } + + boolean running() { + return state.get() == State.RUNNING; + } + + private boolean disabled() { + return state.get() == State.DISABLED; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java index 6a48607977ba9..35f6cfcbcf549 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java @@ -51,7 +51,7 @@ public void start() throws PulsarServerException { } public boolean started() { - return loadManager.started && loadManager.getServiceUnitStateChannel().started(); + return loadManager.running() && loadManager.getServiceUnitStateChannel().started(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index ddbc9eacac921..ce975495feb2a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -255,6 +255,7 @@ public void cancelOwnershipMonitor() { @Override public void cleanOwnerships() { + disable(); doCleanup(brokerId, true); } @@ -412,9 +413,7 @@ public CompletableFuture isChannelOwnerAsync() { if (owner.isPresent()) { return isTargetBroker(owner.get()); } else { - String msg = "There is no channel owner now."; - log.error(msg); - throw new IllegalStateException(msg); + throw new IllegalStateException("There is no channel owner now."); } }); } @@ -679,11 +678,15 @@ private void handleEvent(String serviceUnit, ServiceUnitStateData data) { brokerId, serviceUnit, data, totalHandledRequests); } - if (channelState == Disabled) { + ServiceUnitState state = state(data); + if (channelState == Disabled && (data == null || !data.force())) { + final var request = getOwnerRequests.remove(serviceUnit); + if (request != null) { + request.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException( + "cancel the lookup request for " + serviceUnit + " when receiving " + state)); + } return; } - - ServiceUnitState state = state(data); try { switch (state) { case Owned -> handleOwnEvent(serviceUnit, data); @@ -851,7 +854,7 @@ private void handleSplitEvent(String serviceUnit, ServiceUnitStateData data) { } } - private void handleFreeEvent(String serviceUnit, ServiceUnitStateData data) { + private CompletableFuture handleFreeEvent(String serviceUnit, ServiceUnitStateData data) { var getOwnerRequest = getOwnerRequests.remove(serviceUnit); if (getOwnerRequest != null) { getOwnerRequest.complete(null); @@ -865,8 +868,10 @@ private void handleFreeEvent(String serviceUnit, ServiceUnitStateData data) { : CompletableFuture.completedFuture(0)).thenApply(__ -> null); stateChangeListeners.notifyOnCompletion(future, serviceUnit, data) .whenComplete((__, e) -> log(e, serviceUnit, data, null)); + return future; } else { stateChangeListeners.notify(serviceUnit, data, null); + return CompletableFuture.completedFuture(null); } } @@ -1273,7 +1278,11 @@ private void handleBrokerDeletionEvent(String broker) { return; } } catch (Exception e) { - log.error("Failed to handle broker deletion event.", e); + if (e instanceof ExecutionException && e.getCause() instanceof IllegalStateException) { + log.warn("Failed to handle broker deletion event due to {}", e.getMessage()); + } else { + log.error("Failed to handle broker deletion event.", e); + } return; } MetadataState state = getMetadataState(); @@ -1293,6 +1302,11 @@ private void handleBrokerDeletionEvent(String broker) { private void scheduleCleanup(String broker, long delayInSecs) { var scheduled = new MutableObject>(); try { + final var channelState = this.channelState; + if (channelState == Disabled || channelState == Closed) { + log.warn("[{}] Skip scheduleCleanup because the state is {} now", brokerId, channelState); + return; + } cleanupJobs.computeIfAbsent(broker, k -> { Executor delayed = CompletableFuture .delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor()); @@ -1393,6 +1407,7 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max if (data.state() == Owned && broker.equals(data.dstBroker())) { cleaned = false; + log.info("[{}] bundle {} is still owned by this, data: {}", broker, serviceUnit, data); break; } } @@ -1400,10 +1415,15 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max break; } else { try { - MILLISECONDS.sleep(OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS); + tableview.flush(OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS / 2); + Thread.sleep(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS / 2); } catch (InterruptedException e) { log.warn("Interrupted while delaying the next service unit clean-up. Cleaning broker:{}", brokerId); + } catch (ExecutionException e) { + log.error("Failed to flush table view", e.getCause()); + } catch (TimeoutException e) { + log.warn("Failed to flush the table view in {} ms", OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS); } } } @@ -1428,6 +1448,11 @@ private synchronized void doCleanup(String broker, boolean gracefully) { log.info("Started ownership cleanup for the inactive broker:{}", broker); int orphanServiceUnitCleanupCnt = 0; long totalCleanupErrorCntStart = totalCleanupErrorCnt.get(); + try { + tableview.flush(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS); + } catch (Exception e) { + log.error("Failed to flush", e); + } Map orphanSystemServiceUnits = new HashMap<>(); for (var etr : tableview.entrySet()) { var stateData = etr.getValue(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java index e85134e611632..4a990ddbc9b21 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java @@ -34,7 +34,7 @@ public record ServiceUnitStateData( public ServiceUnitStateData { Objects.requireNonNull(state); - if (StringUtils.isBlank(dstBroker) && StringUtils.isBlank(sourceBroker)) { + if (state != ServiceUnitState.Free && StringUtils.isBlank(dstBroker) && StringUtils.isBlank(sourceBroker)) { throw new IllegalArgumentException("Empty broker"); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java index b1dbb6fac8709..3e43237f4c00e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java @@ -145,4 +145,4 @@ private boolean invalidUnload(ServiceUnitStateData from, ServiceUnitStateData to || !from.dstBroker().equals(to.sourceBroker()) || from.dstBroker().equals(to.dstBroker()); } -} \ No newline at end of file +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java index 8dfaddcdabca1..12cf87445a3dd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java @@ -33,6 +33,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.TopicDomain; @@ -144,8 +145,13 @@ public CompletableFuture put(String key, ServiceUnitStateData value) { .sendAsync() .whenComplete((messageId, e) -> { if (e != null) { - log.error("Failed to publish the message: serviceUnit:{}, data:{}", - key, value, e); + if (e instanceof PulsarClientException.AlreadyClosedException) { + log.info("Skip publishing the message since the producer is closed, serviceUnit: {}, data: " + + "{}", key, value); + } else { + log.error("Failed to publish the message: serviceUnit:{}, data:{}", + key, value, e); + } future.completeExceptionally(e); } else { future.complete(null); @@ -159,7 +165,14 @@ public void flush(long waitDurationInMillis) throws InterruptedException, Timeou if (!isValidState()) { throw new IllegalStateException(INVALID_STATE_ERROR_MSG); } - producer.flushAsync().get(waitDurationInMillis, MILLISECONDS); + final var deadline = System.currentTimeMillis() + waitDurationInMillis; + var waitTimeMs = waitDurationInMillis; + producer.flushAsync().get(waitTimeMs, MILLISECONDS); + waitTimeMs = deadline - System.currentTimeMillis(); + if (waitTimeMs < 0) { + waitTimeMs = 0; + } + tableview.refreshAsync().get(waitTimeMs, MILLISECONDS); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerMaxTopicCountFilter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerMaxTopicCountFilter.java index 48213c18e6376..9863d05ee751e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerMaxTopicCountFilter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerMaxTopicCountFilter.java @@ -41,7 +41,12 @@ public CompletableFuture> filterAsync(Map { - Optional brokerLoadDataOpt = context.brokerLoadDataStore().get(broker); + final Optional brokerLoadDataOpt; + try { + brokerLoadDataOpt = context.brokerLoadDataStore().get(broker); + } catch (IllegalStateException ignored) { + return false; + } long topics = brokerLoadDataOpt.map(BrokerLoadData::getTopics).orElse(0L); // TODO: The broker load data might be delayed, so the max topic check might not accurate. return topics >= loadBalancerBrokerMaxTopics; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java index c9d18676cfa99..3ce44a1e65a73 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java @@ -92,7 +92,11 @@ public synchronized CompletableFuture removeAsync(String key) { public synchronized Optional get(String key) { String msg = validateTableView(); if (StringUtils.isNotBlank(msg)) { - throw new IllegalStateException(msg); + if (msg.equals(SHUTDOWN_ERR_MSG)) { + return Optional.empty(); + } else { + throw new IllegalStateException(msg); + } } return Optional.ofNullable(tableView.get(key)); } @@ -193,7 +197,9 @@ public synchronized void startProducer() throws LoadDataStoreException { @Override public synchronized void close() throws IOException { - validateState(); + if (isShutdown) { + return; + } closeProducer(); closeTableView(); } 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 09f04d878c4e5..bfa99eedcadce 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 @@ -973,7 +973,12 @@ public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean cl pulsar.getNamespaceService().unloadNamespaceBundle(su, timeout, MILLISECONDS, closeWithoutWaitingClientDisconnect).get(timeout, MILLISECONDS); } catch (Exception e) { - log.warn("Failed to unload namespace bundle {}", su, e); + if (e instanceof ExecutionException + && e.getCause() instanceof ServiceUnitNotReadyException) { + log.warn("Failed to unload namespace bundle {}: {}", su, e.getMessage()); + } else { + log.warn("Failed to unload namespace bundle {}", su, e); + } } } }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 18b4c610a5c9b..6ff6408916b1c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -254,7 +254,7 @@ public CompletableFuture> getTopicPoliciesAsync(TopicNam // initialization by calling this method. At the moment, the load manager does not start so the lookup // for "__change_events" will fail. In this case, just return an empty policies to avoid deadlock. final var loadManager = pulsarService.getLoadManager().get(); - if (loadManager == null || !loadManager.started()) { + if (loadManager == null || !loadManager.started() || closed.get()) { return CompletableFuture.completedFuture(Optional.empty()); } final CompletableFuture preparedFuture = prepareInitPoliciesCacheAsync(topicName.getNamespaceObject()); @@ -308,6 +308,9 @@ public void addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { @VisibleForTesting @Nonnull CompletableFuture prepareInitPoliciesCacheAsync(@Nonnull NamespaceName namespace) { requireNonNull(namespace); + if (closed.get()) { + return CompletableFuture.completedFuture(false); + } return pulsarService.getPulsarResources().getNamespaceResources().getPoliciesAsync(namespace) .thenCompose(namespacePolicies -> { if (namespacePolicies.isEmpty() || namespacePolicies.get().deleted) { @@ -331,6 +334,9 @@ public void addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { }); initFuture.exceptionally(ex -> { try { + if (closed.get()) { + return null; + } log.error("[{}] Failed to create reader on __change_events topic", namespace, ex); cleanCacheAndCloseReader(namespace, false); @@ -681,14 +687,22 @@ public void close() throws Exception { if (closed.compareAndSet(false, true)) { writerCaches.synchronous().invalidateAll(); readerCaches.values().forEach(future -> { - if (future != null && !future.isCompletedExceptionally()) { - future.thenAccept(reader -> { - try { - reader.close(); - } catch (Exception e) { - log.error("Failed to close reader.", e); - } - }); + try { + final var reader = future.getNow(null); + if (reader != null) { + reader.close(); + log.info("Closed the reader for topic policies"); + } else { + // Avoid blocking the thread that the reader is created + future.thenAccept(SystemTopicClient.Reader::closeAsync).whenComplete((__, e) -> { + if (e == null) { + log.info("Closed the reader for topic policies"); + } else { + log.error("Failed to close the reader for topic policies", e); + } + }); + } + } catch (Throwable ignored) { } }); readerCaches.clear(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java index 41413f3e3a913..fa63ce566c603 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java @@ -22,13 +22,15 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -36,26 +38,33 @@ import org.testng.annotations.Test; @Slf4j +@Test(groups = "broker") public class ExtensibleLoadManagerCloseTest { private static final String clusterName = "test"; - private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(1, 0, () -> 0); private final List brokers = new ArrayList<>(); - private PulsarAdmin admin; + private LocalBookkeeperEnsemble bk; @BeforeClass(alwaysRun = true) public void setup() throws Exception { + bk = new LocalBookkeeperEnsemble(1, 0, () -> 0); bk.start(); - for (int i = 0; i < 3; i++) { + } + + private void setupBrokers(int numBrokers) throws Exception { + brokers.clear(); + for (int i = 0; i < numBrokers; i++) { final var broker = new PulsarService(brokerConfig()); broker.start(); brokers.add(broker); } - admin = brokers.get(0).getAdminClient(); - admin.clusters().createCluster(clusterName, ClusterData.builder().build()); - admin.tenants().createTenant("public", TenantInfo.builder() - .allowedClusters(Collections.singleton(clusterName)).build()); - admin.namespaces().createNamespace("public/default"); + final var admin = brokers.get(0).getAdminClient(); + if (!admin.clusters().getClusters().contains(clusterName)) { + admin.clusters().createCluster(clusterName, ClusterData.builder().build()); + admin.tenants().createTenant("public", TenantInfo.builder() + .allowedClusters(Collections.singleton(clusterName)).build()); + admin.namespaces().createNamespace("public/default"); + } } @@ -85,7 +94,9 @@ private ServiceConfiguration brokerConfig() { @Test public void testCloseAfterLoadingBundles() throws Exception { + setupBrokers(3); final var topic = "test"; + final var admin = brokers.get(0).getAdminClient(); admin.topics().createPartitionedTopic(topic, 20); admin.lookups().lookupPartitionedTopic(topic); final var client = PulsarClient.builder().serviceUrl(brokers.get(0).getBrokerServiceUrl()).build(); @@ -104,4 +115,25 @@ public void testCloseAfterLoadingBundles() throws Exception { Assert.assertTrue(closeTimeMs < 5000L); } } + + @Test + public void testLookup() throws Exception { + setupBrokers(1); + final var topic = "test-lookup"; + final var numPartitions = 16; + final var admin = brokers.get(0).getAdminClient(); + admin.topics().createPartitionedTopic(topic, numPartitions); + + final var futures = new ArrayList>(); + for (int i = 0; i < numPartitions; i++) { + futures.add(admin.lookups().lookupTopicAsync(topic + TopicName.PARTITIONED_TOPIC_SUFFIX + i)); + } + FutureUtil.waitForAll(futures).get(); + + final var start = System.currentTimeMillis(); + brokers.get(0).close(); + final var closeTimeMs = System.currentTimeMillis() - start; + log.info("Broker close time: {}", closeTimeMs); + Assert.assertTrue(closeTimeMs < 5000L); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java index 3267e67ad2c3e..820307637be67 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java @@ -198,7 +198,7 @@ public void testShutdown() throws Exception { Assert.assertTrue(loadDataStore.pushAsync("2", 2).isCompletedExceptionally()); Assert.assertTrue(loadDataStore.removeAsync("2").isCompletedExceptionally()); - assertThrows(IllegalStateException.class, () -> loadDataStore.get("2")); + assertTrue(loadDataStore.get("2").isEmpty()); assertThrows(IllegalStateException.class, loadDataStore::size); assertThrows(IllegalStateException.class, loadDataStore::entrySet); assertThrows(IllegalStateException.class, () -> loadDataStore.forEach((k, v) -> {})); @@ -206,7 +206,6 @@ public void testShutdown() throws Exception { assertThrows(IllegalStateException.class, loadDataStore::start); assertThrows(IllegalStateException.class, loadDataStore::startProducer); assertThrows(IllegalStateException.class, loadDataStore::startTableView); - assertThrows(IllegalStateException.class, loadDataStore::close); assertThrows(IllegalStateException.class, loadDataStore::closeTableView); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java index d5d4174ee10a9..4f52060497864 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java @@ -364,8 +364,8 @@ private void readAllExistingMessages(Reader reader, CompletableFuture f } }).exceptionally(ex -> { if (ex.getCause() instanceof PulsarClientException.AlreadyClosedException) { - log.error("Reader {} was closed while reading existing messages.", - reader.getTopic(), ex); + log.info("Reader {} was closed while reading existing messages.", + reader.getTopic()); } else { log.warn("Reader {} was interrupted while reading existing messages. ", reader.getTopic(), ex); @@ -393,8 +393,7 @@ private void readTailMessages(Reader reader) { readTailMessages(reader); }).exceptionally(ex -> { if (ex.getCause() instanceof PulsarClientException.AlreadyClosedException) { - log.error("Reader {} was closed while reading tail messages.", - reader.getTopic(), ex); + log.info("Reader {} was closed while reading tail messages.", reader.getTopic()); // Fail all refresh request when no more messages can be read. pendingRefreshRequests.keySet().forEach(future -> { pendingRefreshRequests.remove(future); From 95bd1d1dd3d447f0705a96092afbc9d6bd6cd1dc Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 27 Sep 2024 19:49:02 +0800 Subject: [PATCH 012/327] [fix][broker] Fix the broker registery cannot recover from the metadata node deletion (#23359) --- .../extensions/BrokerRegistry.java | 6 +- .../extensions/BrokerRegistryImpl.java | 57 ++++---- .../BrokerRegistryIntegrationTest.java | 124 ++++++++++++++++++ .../extensions/BrokerRegistryTest.java | 6 +- .../ExtensibleLoadManagerImplTest.java | 4 +- 5 files changed, 168 insertions(+), 29 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java index 8133d4c482752..79dba9c63342e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java @@ -25,6 +25,8 @@ import java.util.function.BiConsumer; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; +import org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.NotificationType; @@ -32,6 +34,8 @@ * Responsible for registering the current Broker lookup info to * the distributed store (e.g. Zookeeper) for broker discovery. */ +@InterfaceAudience.LimitedPrivate +@InterfaceStability.Unstable public interface BrokerRegistry extends AutoCloseable { /** @@ -47,7 +51,7 @@ public interface BrokerRegistry extends AutoCloseable { /** * Register local broker to metadata store. */ - void register() throws MetadataStoreException; + CompletableFuture registerAsync(); /** * Unregister the broker. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java index f34d377990b68..9fd0518a054cc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java @@ -32,6 +32,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarServerException; @@ -69,10 +70,12 @@ protected enum State { Init, Started, Registered, + Unregistering, Closed } - private State state; + @VisibleForTesting + final AtomicReference state = new AtomicReference<>(State.Init); public BrokerRegistryImpl(PulsarService pulsar) { this.pulsar = pulsar; @@ -80,6 +83,13 @@ public BrokerRegistryImpl(PulsarService pulsar) { this.brokerLookupDataMetadataCache = pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class); this.scheduler = pulsar.getLoadManagerExecutor(); this.listeners = new ArrayList<>(); + // The registered node is an ephemeral node that could be deleted when the metadata store client's session + // is expired. In this case, we should register again. + this.listeners.add((broker, notificationType) -> { + if (notificationType == NotificationType.Deleted && getBrokerId().equals(broker)) { + registerAsync(); + } + }); this.brokerIdKeyPath = keyPath(pulsar.getBrokerId()); this.brokerLookupData = new BrokerLookupData( pulsar.getWebServiceAddress(), @@ -94,44 +104,45 @@ public BrokerRegistryImpl(PulsarService pulsar) { System.currentTimeMillis(), pulsar.getBrokerVersion(), pulsar.getConfig().lookupProperties()); - this.state = State.Init; } @Override public synchronized void start() throws PulsarServerException { - if (this.state != State.Init) { - return; + if (!this.state.compareAndSet(State.Init, State.Started)) { + throw new PulsarServerException("Cannot start the broker registry in state " + state.get()); } pulsar.getLocalMetadataStore().registerListener(this::handleMetadataStoreNotification); try { - this.state = State.Started; - this.register(); - } catch (MetadataStoreException e) { - throw new PulsarServerException(e); + this.registerAsync().get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + throw PulsarServerException.from(e); } } @Override public boolean isStarted() { - return this.state == State.Started || this.state == State.Registered; + final var state = this.state.get(); + return state == State.Started || state == State.Registered; } @Override - public synchronized void register() throws MetadataStoreException { - if (this.state == State.Started) { - try { - brokerLookupDataMetadataCache.put(brokerIdKeyPath, brokerLookupData, EnumSet.of(CreateOption.Ephemeral)) - .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); - this.state = State.Registered; - } catch (InterruptedException | ExecutionException | TimeoutException e) { - throw MetadataStoreException.unwrap(e); - } + public CompletableFuture registerAsync() { + final var state = this.state.get(); + if (state != State.Started && state != State.Registered) { + log.info("[{}] Skip registering self because the state is {}", getBrokerId(), state); + return CompletableFuture.completedFuture(null); } + log.info("[{}] Started registering self to {} (state: {})", getBrokerId(), brokerIdKeyPath, state); + return brokerLookupDataMetadataCache.put(brokerIdKeyPath, brokerLookupData, EnumSet.of(CreateOption.Ephemeral)) + .thenAccept(__ -> { + this.state.set(State.Registered); + log.info("[{}] Finished registering self", getBrokerId()); + }); } @Override public synchronized void unregister() throws MetadataStoreException { - if (this.state == State.Registered) { + if (state.compareAndSet(State.Registered, State.Unregistering)) { try { brokerLookupDataMetadataCache.delete(brokerIdKeyPath) .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); @@ -144,7 +155,7 @@ public synchronized void unregister() throws MetadataStoreException { } catch (InterruptedException | TimeoutException e) { throw MetadataStoreException.unwrap(e); } finally { - this.state = State.Started; + state.set(State.Started); } } } @@ -191,7 +202,7 @@ public synchronized void addListener(BiConsumer listen @Override public synchronized void close() throws PulsarServerException { - if (this.state == State.Closed) { + if (this.state.get() == State.Closed) { return; } try { @@ -200,7 +211,7 @@ public synchronized void close() throws PulsarServerException { } catch (Exception ex) { log.error("Unexpected error when unregistering the broker registry", ex); } finally { - this.state = State.Closed; + this.state.set(State.Closed); } } @@ -238,7 +249,7 @@ protected static String keyPath(String brokerId) { } private void checkState() throws IllegalStateException { - if (this.state == State.Closed) { + if (this.state.get() == State.Closed) { throw new IllegalStateException("The registry already closed."); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java new file mode 100644 index 0000000000000..162ea50829d40 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -0,0 +1,124 @@ +/* + * 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.loadbalance.extensions; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.util.PortManager; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.LoadManager; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.awaitility.Awaitility; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class BrokerRegistryIntegrationTest { + + private static final String clusterName = "test"; + private final int zkPort = PortManager.nextFreePort(); + private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(2, zkPort, PortManager::nextFreePort); + private PulsarService pulsar; + private BrokerRegistry brokerRegistry; + private String brokerMetadataPath; + + @BeforeClass + protected void setup() throws Exception { + bk.start(); + pulsar = new PulsarService(brokerConfig()); + pulsar.start(); + final var admin = pulsar.getAdminClient(); + admin.clusters().createCluster(clusterName, ClusterData.builder().build()); + admin.tenants().createTenant("public", TenantInfo.builder() + .allowedClusters(Collections.singleton(clusterName)).build()); + admin.namespaces().createNamespace("public/default"); + brokerRegistry = ((ExtensibleLoadManagerWrapper) pulsar.getLoadManager().get()).get().getBrokerRegistry(); + brokerMetadataPath = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + pulsar.getBrokerId(); + } + + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + if (pulsar != null) { + pulsar.close(); + } + bk.stop(); + } + + @Test + public void testRecoverFromNodeDeletion() throws Exception { + // Simulate the case that the node was somehow deleted (e.g. by session timeout) + Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( + brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); + pulsar.getLocalMetadataStore().delete(brokerMetadataPath, Optional.empty()); + Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( + brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); + + // If the node is deleted by unregister(), it should not recreate the path + brokerRegistry.unregister(); + Thread.sleep(3000); + Assert.assertTrue(brokerRegistry.getAvailableBrokersAsync().get().isEmpty()); + + // Restore the normal state + brokerRegistry.registerAsync().get(); + Assert.assertEquals(brokerRegistry.getAvailableBrokersAsync().get(), List.of(pulsar.getBrokerId())); + } + + @Test + public void testRegisterAgain() throws Exception { + Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( + brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); + final var metadataStore = pulsar.getLocalMetadataStore(); + final var oldResult = metadataStore.get(brokerMetadataPath).get().orElseThrow(); + log.info("Old result: {} {}", new String(oldResult.getValue()), oldResult.getStat().getVersion()); + brokerRegistry.registerAsync().get(); + + Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> { + final var newResult = metadataStore.get(brokerMetadataPath).get().orElseThrow(); + log.info("New result: {} {}", new String(newResult.getValue()), newResult.getStat().getVersion()); + Assert.assertTrue(newResult.getStat().getVersion() > oldResult.getStat().getVersion()); + Assert.assertEquals(newResult.getValue(), oldResult.getValue()); + }); + } + + private ServiceConfiguration brokerConfig() { + final var config = new ServiceConfiguration(); + config.setClusterName(clusterName); + config.setAdvertisedAddress("localhost"); + config.setBrokerServicePort(Optional.of(0)); + config.setWebServicePort(Optional.of(0)); + config.setMetadataStoreUrl("zk:127.0.0.1:" + bk.getZookeeperPort()); + config.setManagedLedgerDefaultWriteQuorum(1); + config.setManagedLedgerDefaultAckQuorum(1); + config.setManagedLedgerDefaultEnsembleSize(1); + config.setDefaultNumberOfNamespaceBundles(16); + config.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + config.setLoadBalancerDebugModeEnabled(true); + config.setBrokerShutdownTimeoutMs(100); + return config; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java index 91ada90dda690..28a2a18500f5f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java @@ -332,7 +332,7 @@ public void testCloseRegister() throws Exception { assertEquals(getState(brokerRegistry), BrokerRegistryImpl.State.Started); // Check state after re-register. - brokerRegistry.register(); + brokerRegistry.registerAsync().get(); assertEquals(getState(brokerRegistry), BrokerRegistryImpl.State.Registered); // Check state after close. @@ -396,8 +396,8 @@ public void testKeyPath() { assertEquals(keyPath, LOADBALANCE_BROKERS_ROOT + "/brokerId"); } - public BrokerRegistryImpl.State getState(BrokerRegistryImpl brokerRegistry) { - return WhiteboxImpl.getInternalState(brokerRegistry, BrokerRegistryImpl.State.class); + private static BrokerRegistryImpl.State getState(BrokerRegistryImpl brokerRegistry) { + return brokerRegistry.state.get(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 4f6a006918318..7871e612c847a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -1211,7 +1211,7 @@ public void testDeployAndRollbackLoadManager() throws Exception { producer.send("t1"); // Test re-register broker and check the lookup result - loadManager4.getBrokerRegistry().register(); + loadManager4.getBrokerRegistry().registerAsync().get(); result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); assertNotNull(result); @@ -1423,7 +1423,7 @@ public void testLoadBalancerServiceUnitTableViewSyncer() throws Exception { producer.send("t1"); // Test re-register broker and check the lookup result - loadManager4.getBrokerRegistry().register(); + loadManager4.getBrokerRegistry().registerAsync().get(); result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); assertNotNull(result); From ca4a7435db6a99560284324b470ff66cae9d84f5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 28 Sep 2024 20:10:54 +0300 Subject: [PATCH 013/327] [fix][build] Disable flaky test BrokerRegistryIntegrationTest (#23367) --- .../extensions/BrokerRegistryIntegrationTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java index 162ea50829d40..d6615a8a5b49b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -37,7 +37,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "broker") +@Test(groups = "flaky") public class BrokerRegistryIntegrationTest { private static final String clusterName = "test"; @@ -69,7 +69,7 @@ protected void cleanup() throws Exception { bk.stop(); } - @Test + @Test(enabled = false) public void testRecoverFromNodeDeletion() throws Exception { // Simulate the case that the node was somehow deleted (e.g. by session timeout) Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( @@ -88,7 +88,7 @@ public void testRecoverFromNodeDeletion() throws Exception { Assert.assertEquals(brokerRegistry.getAvailableBrokersAsync().get(), List.of(pulsar.getBrokerId())); } - @Test + @Test(enabled = false) public void testRegisterAgain() throws Exception { Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); From 77cb67a8c05210b9af0deb719cd24e3c3f5521b1 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Sun, 29 Sep 2024 11:16:33 +0800 Subject: [PATCH 014/327] [fix][ml] Managed ledger should recover after open ledger failed (#23368) --- .../impl/ManagedLedgerFactoryImpl.java | 1 + .../mledger/impl/ManagedLedgerErrorsTest.java | 31 +++++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 34dd3610d4ec9..f546a487f84be 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -440,6 +440,7 @@ public void initializeFailed(ManagedLedgerException e) { // Clean the map if initialization fails ledgers.remove(name, future); + entryCacheManager.removeEntryCache(name); if (pendingInitializeLedgers.remove(name, pendingLedger)) { pendingLedger.ledger.asyncClose(new CloseCallback() { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java index 7b2f8228ad722..d72bffa27d30a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java @@ -31,12 +31,14 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; import org.apache.bookkeeper.mledger.Entry; @@ -509,6 +511,35 @@ public void recoverAfterWriteError() throws Exception { entries.forEach(Entry::release); } + @Test + public void recoverAfterOpenManagedLedgerFail() throws Exception { + ManagedLedger ledger = factory.open("recoverAfterOpenManagedLedgerFail"); + Position position = ledger.addEntry("entry".getBytes()); + ledger.close(); + bkc.failAfter(0, BKException.Code.BookieHandleNotAvailableException); + try { + factory.open("recoverAfterOpenManagedLedgerFail"); + } catch (Exception e) { + // ok + } + + ledger = factory.open("recoverAfterOpenManagedLedgerFail"); + CompletableFuture future = new CompletableFuture<>(); + ledger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + future.complete(entry.getData()); + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null); + byte[] bytes = future.get(30, TimeUnit.SECONDS); + assertEquals(new String(bytes), "entry"); + } + @Test public void recoverLongTimeAfterMultipleWriteErrors() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("recoverLongTimeAfterMultipleWriteErrors"); From 950309b57a3674c43e2168168cdb9ef670ac6274 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sun, 29 Sep 2024 06:28:06 +0300 Subject: [PATCH 015/327] [fix][misc] Log Conscrypt security provider initialization warnings at debug level (#23364) --- .../org/apache/pulsar/common/util/SecurityUtility.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java index 8c1f1f5d8b39c..2b7b1a984634f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java @@ -133,12 +133,12 @@ private static Provider loadConscryptProvider() { conscryptClazz.getMethod("checkAvailability").invoke(null); } catch (Throwable e) { if (e instanceof ClassNotFoundException) { - log.warn("Conscrypt isn't available in the classpath. Using JDK default security provider."); + log.debug("Conscrypt isn't available in the classpath. Using JDK default security provider."); } else if (e.getCause() instanceof UnsatisfiedLinkError) { - log.warn("Conscrypt isn't available for {} {}. Using JDK default security provider.", + log.debug("Conscrypt isn't available for {} {}. Using JDK default security provider.", System.getProperty("os.name"), System.getProperty("os.arch")); } else { - log.warn("Conscrypt isn't available. Using JDK default security provider." + log.debug("Conscrypt isn't available. Using JDK default security provider." + " Cause : {}, Reason : {}", e.getCause(), e.getMessage()); } return null; @@ -148,7 +148,7 @@ private static Provider loadConscryptProvider() { try { provider = (Provider) Class.forName(CONSCRYPT_PROVIDER_CLASS).getDeclaredConstructor().newInstance(); } catch (ReflectiveOperationException e) { - log.warn("Unable to get security provider for class {}", CONSCRYPT_PROVIDER_CLASS, e); + log.debug("Unable to get security provider for class {}", CONSCRYPT_PROVIDER_CLASS, e); return null; } From 5e832a1cc1441eaf8d64fe72c1a2af8829030d3d Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Sun, 29 Sep 2024 15:13:38 +0800 Subject: [PATCH 016/327] [fix][test] Fix ReplicatorRateLimiterTest (#23369) --- .../apache/pulsar/broker/service/ReplicatorRateLimiterTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java index bec6b558ea401..2e0dd0a90e8a6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java @@ -611,7 +611,7 @@ public void testReplicatorRateLimiterByBytes() throws Exception { } private static Optional getRateLimiter(PersistentTopic topic) { - return getRateLimiter(topic); + return topic.getReplicators().values().stream().findFirst().map(Replicator::getRateLimiter).orElseThrow(); } private static final Logger log = LoggerFactory.getLogger(ReplicatorRateLimiterTest.class); From f071a898775de3b527b538477d94a326f4b9c7e8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sun, 29 Sep 2024 14:16:54 +0300 Subject: [PATCH 017/327] [fix][cli] Remove deprecated "-client" JVM arg (#23370) --- conf/pulsar_tools_env.sh | 3 --- 1 file changed, 3 deletions(-) diff --git a/conf/pulsar_tools_env.sh b/conf/pulsar_tools_env.sh index 9d22b73905df3..96ee304bf0b3a 100755 --- a/conf/pulsar_tools_env.sh +++ b/conf/pulsar_tools_env.sh @@ -57,9 +57,6 @@ if [ -n "$PULSAR_MEM" ]; then fi PULSAR_MEM=${PULSAR_MEM:-"-Xmx128m -XX:MaxDirectMemorySize=128m"} -# Garbage collection options -PULSAR_GC=${PULSAR_GC:-" -client "} - # Extra options to be passed to the jvm PULSAR_EXTRA_OPTS="${PULSAR_MEM} ${PULSAR_GC} ${PULSAR_GC_LOG} -Dio.netty.leakDetectionLevel=disabled ${PULSAR_EXTRA_OPTS}" From 7e59bdeb9d142430c7119346a34c488045271c19 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sun, 29 Sep 2024 20:09:54 +0300 Subject: [PATCH 018/327] [improve] Install openssl in the docker image to fix compatibility with Apache Pulsar Helm chart (#23362) --- docker/pulsar/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 81446ae5ee5ce..38b74e8503d51 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -92,7 +92,8 @@ RUN apk add --no-cache \ ca-certificates \ procps \ curl \ - bind-tools + bind-tools \ + openssl # Upgrade all packages to get latest versions with security fixes RUN apk upgrade --no-cache From b24285029b1113840ded42404229bc0eb344d5bd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 30 Sep 2024 07:09:04 +0300 Subject: [PATCH 019/327] [improve][ci] Switch to Java 21 as default JVM version for CI (#23373) --- .github/workflows/pulsar-ci.yaml | 14 +++++++------- bin/function-localrunner | 9 ++++++++- conf/bkenv.sh | 14 +++++++++++--- conf/pulsar_env.sh | 13 ++++++++++--- docker/pulsar/Dockerfile | 3 ++- docker/pulsar/pom.xml | 1 + pom.xml | 28 +--------------------------- 7 files changed, 40 insertions(+), 42 deletions(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index dd93003eecce6..ad017674ac6ee 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -25,9 +25,9 @@ on: - branch-* - pulsar-* schedule: - # scheduled job with JDK 17 - - cron: '0 12 * * *' # scheduled job with JDK 21 + - cron: '0 12 * * *' + # scheduled job with JDK 17 # if cron expression is changed, make sure to update the expression in jdk_major_version step in preconditions job - cron: '0 6 * * *' workflow_dispatch: @@ -44,7 +44,7 @@ on: options: - '17' - '21' - default: '17' + default: '21' trace_test_resource_cleanup: description: 'Collect thread & heap information before exiting a test JVM. When set to "on", thread dump and heap histogram will be collected. When set to "full", a heap dump will also be collected.' required: true @@ -95,13 +95,13 @@ jobs: - name: Select JDK major version id: jdk_major_version run: | - # use JDK 21 for the scheduled build with cron expression '0 6 * * *' + # use JDK 17 for the scheduled build with cron expression '0 6 * * *' if [[ "${{ github.event_name == 'schedule' && github.event.schedule == '0 6 * * *' && 'true' || 'false' }}" == "true" ]]; then - echo "jdk_major_version=21" >> $GITHUB_OUTPUT + echo "jdk_major_version=17" >> $GITHUB_OUTPUT exit 0 fi - # use JDK 17 for build unless overridden with workflow_dispatch input - echo "jdk_major_version=${{ github.event_name == 'workflow_dispatch' && github.event.inputs.jdk_major_version || '17'}}" >> $GITHUB_OUTPUT + # use JDK 21 for build unless overridden with workflow_dispatch input + echo "jdk_major_version=${{ github.event_name == 'workflow_dispatch' && github.event.inputs.jdk_major_version || '21'}}" >> $GITHUB_OUTPUT - name: checkout if: ${{ github.event_name == 'pull_request' }} diff --git a/bin/function-localrunner b/bin/function-localrunner index b2405db724e72..a47f3efa48609 100755 --- a/bin/function-localrunner +++ b/bin/function-localrunner @@ -52,7 +52,14 @@ done PULSAR_MEM=${PULSAR_MEM:-"-Xmx128m -XX:MaxDirectMemorySize=128m"} # Garbage collection options -PULSAR_GC=${PULSAR_GC:-"-XX:+UseZGC -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch"} +if [ -z "$PULSAR_GC" ]; then + PULSAR_GC="-XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch" + if [[ $JAVA_MAJOR_VERSION -ge 21 ]]; then + PULSAR_GC="-XX:+UseZGC -XX:+ZGenerational ${PULSAR_GC}" + else + PULSAR_GC="-XX:+UseZGC ${PULSAR_GC}" + fi +fi # Garbage collection log. PULSAR_GC_LOG_DIR=${PULSAR_GC_LOG_DIR:-logs} diff --git a/conf/bkenv.sh b/conf/bkenv.sh index b41532d3a0c91..8beea47cee312 100644 --- a/conf/bkenv.sh +++ b/conf/bkenv.sh @@ -37,9 +37,6 @@ BOOKIE_LOG_DIR=${BOOKIE_LOG_DIR:-"${PULSAR_LOG_DIR}"} # Memory size options BOOKIE_MEM=${BOOKIE_MEM:-${PULSAR_MEM:-"-Xms2g -Xmx2g -XX:MaxDirectMemorySize=2g"}} -# Garbage collection options -BOOKIE_GC=${BOOKIE_GC:-${PULSAR_GC:-"-XX:+UseZGC -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch"}} - if [ -z "$JAVA_HOME" ]; then JAVA_BIN=java else @@ -60,6 +57,17 @@ for token in $("$JAVA_BIN" -version 2>&1 | grep 'version "'); do fi done +# Garbage collection options +BOOKIE_GC="${BOOKIE_GC:-${PULSAR_GC}}" +if [ -z "$BOOKIE_GC" ]; then + BOOKIE_GC="-XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch" + if [[ $JAVA_MAJOR_VERSION -ge 21 ]]; then + BOOKIE_GC="-XX:+UseZGC -XX:+ZGenerational ${BOOKIE_GC}" + else + BOOKIE_GC="-XX:+UseZGC ${BOOKIE_GC}" + fi +fi + if [[ -z "$BOOKIE_GC_LOG" ]]; then # fallback to PULSAR_GC_LOG if it is set BOOKIE_GC_LOG="$PULSAR_GC_LOG" diff --git a/conf/pulsar_env.sh b/conf/pulsar_env.sh index 3a069e31fdc90..f95d0ac83c13a 100755 --- a/conf/pulsar_env.sh +++ b/conf/pulsar_env.sh @@ -44,9 +44,6 @@ # Extra options to be passed to the jvm PULSAR_MEM=${PULSAR_MEM:-"-Xms2g -Xmx2g -XX:MaxDirectMemorySize=4g"} -# Garbage collection options -PULSAR_GC=${PULSAR_GC:-"-XX:+UseZGC -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch"} - if [ -z "$JAVA_HOME" ]; then JAVA_BIN=java else @@ -67,6 +64,16 @@ for token in $("$JAVA_BIN" -version 2>&1 | grep 'version "'); do fi done +# Garbage collection options +if [ -z "$PULSAR_GC" ]; then + PULSAR_GC="-XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch" + if [[ $JAVA_MAJOR_VERSION -ge 21 ]]; then + PULSAR_GC="-XX:+UseZGC -XX:+ZGenerational ${PULSAR_GC}" + else + PULSAR_GC="-XX:+UseZGC ${PULSAR_GC}" + fi +fi + PULSAR_GC_LOG_DIR=${PULSAR_GC_LOG_DIR:-"${PULSAR_LOG_DIR}"} if [[ -z "$PULSAR_GC_LOG" ]]; then diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 38b74e8503d51..f3b0f3d944bdc 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -18,6 +18,7 @@ # ARG ALPINE_VERSION=3.20 +ARG IMAGE_JDK_MAJOR_VERSION=21 # First create a stage with just the Pulsar tarball and scripts FROM alpine:$ALPINE_VERSION as pulsar @@ -54,7 +55,7 @@ RUN chmod -R o+rx /pulsar RUN echo 'OPTS="$OPTS -Dorg.xerial.snappy.use.systemlib=true"' >> /pulsar/conf/bkenv.sh ### Create one stage to include JVM distribution -FROM amazoncorretto:21-alpine AS jvm +FROM amazoncorretto:${IMAGE_JDK_MAJOR_VERSION}-alpine AS jvm RUN apk add --no-cache binutils diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 68d82ae552825..481fc319be732 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -83,6 +83,7 @@ target/pulsar-server-distribution-${project.version}-bin.tar.gz ${pulsar.client.python.version} ${snappy.version} + ${IMAGE_JDK_MAJOR_VERSION} ${project.basedir} diff --git a/pom.xml b/pom.xml index 1f7ecd3b90c1e..9f2330d7c75e1 100644 --- a/pom.xml +++ b/pom.xml @@ -84,9 +84,7 @@ flexible messaging model and an intuitive client API. 3.4.0 - http://archive.ubuntu.com/ubuntu/ - http://security.ubuntu.com/ubuntu/ - 17 + 21 **/Test*.java,**/*Test.java,**/*Tests.java,**/*TestCase.java @@ -2748,30 +2746,6 @@ flexible messaging model and an intuitive client API. - - ubuntu-mirror-set - - - env.UBUNTU_MIRROR - - - - - ${env.UBUNTU_MIRROR} - - - - ubuntu-security-mirror-set - - - env.UBUNTU_SECURITY_MIRROR - - - - - ${env.UBUNTU_SECURITY_MIRROR} - - jdk-major-version-set From 7d7dc80f0ee9e5926c8ac53a155de98bc6ffa3fa Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 30 Sep 2024 14:20:33 +0800 Subject: [PATCH 020/327] [fix][broker] Fix the broker registering might be blocked for long time (#23371) --- .../extensions/BrokerRegistryImpl.java | 14 ++++---- .../BrokerRegistryIntegrationTest.java | 13 ++++--- ...rRegistryMetadataStoreIntegrationTest.java | 35 +++++++++++++++++++ .../pulsar/client/impl/TableViewTest.java | 3 ++ .../pulsar/client/impl/TableViewImpl.java | 25 +++++++++---- 5 files changed, 71 insertions(+), 19 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java index 9fd0518a054cc..a13b332e6eb5f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java @@ -83,13 +83,6 @@ public BrokerRegistryImpl(PulsarService pulsar) { this.brokerLookupDataMetadataCache = pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class); this.scheduler = pulsar.getLoadManagerExecutor(); this.listeners = new ArrayList<>(); - // The registered node is an ephemeral node that could be deleted when the metadata store client's session - // is expired. In this case, we should register again. - this.listeners.add((broker, notificationType) -> { - if (notificationType == NotificationType.Deleted && getBrokerId().equals(broker)) { - registerAsync(); - } - }); this.brokerIdKeyPath = keyPath(pulsar.getBrokerId()); this.brokerLookupData = new BrokerLookupData( pulsar.getWebServiceAddress(), @@ -223,11 +216,16 @@ private void handleMetadataStoreNotification(Notification t) { if (log.isDebugEnabled()) { log.debug("Handle notification: [{}]", t); } + // The registered node is an ephemeral node that could be deleted when the metadata store client's session + // is expired. In this case, we should register again. + final var brokerId = t.getPath().substring(LOADBALANCE_BROKERS_ROOT.length() + 1); + if (t.getType() == NotificationType.Deleted && getBrokerId().equals(brokerId)) { + registerAsync(); + } if (listeners.isEmpty()) { return; } this.scheduler.submit(() -> { - String brokerId = t.getPath().substring(LOADBALANCE_BROKERS_ROOT.length() + 1); for (BiConsumer listener : listeners) { listener.accept(brokerId, t.getType()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java index d6615a8a5b49b..232088afb94fe 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -37,7 +37,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "flaky") +@Test(groups = "broker") public class BrokerRegistryIntegrationTest { private static final String clusterName = "test"; @@ -63,13 +63,18 @@ protected void setup() throws Exception { @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { + final var startMs = System.currentTimeMillis(); if (pulsar != null) { pulsar.close(); } + final var elapsedMs = System.currentTimeMillis() - startMs; bk.stop(); + if (elapsedMs > 5000) { + throw new RuntimeException("Broker took " + elapsedMs + "ms to close"); + } } - @Test(enabled = false) + @Test public void testRecoverFromNodeDeletion() throws Exception { // Simulate the case that the node was somehow deleted (e.g. by session timeout) Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( @@ -88,7 +93,7 @@ public void testRecoverFromNodeDeletion() throws Exception { Assert.assertEquals(brokerRegistry.getAvailableBrokersAsync().get(), List.of(pulsar.getBrokerId())); } - @Test(enabled = false) + @Test public void testRegisterAgain() throws Exception { Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); @@ -105,7 +110,7 @@ public void testRegisterAgain() throws Exception { }); } - private ServiceConfiguration brokerConfig() { + protected ServiceConfiguration brokerConfig() { final var config = new ServiceConfiguration(); config.setClusterName(clusterName); config.setAdvertisedAddress("localhost"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java new file mode 100644 index 0000000000000..3e01b1fad0f21 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java @@ -0,0 +1,35 @@ +/* + * 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.loadbalance.extensions; + +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class BrokerRegistryMetadataStoreIntegrationTest extends BrokerRegistryIntegrationTest { + + @Override + protected ServiceConfiguration brokerConfig() { + final var config = super.brokerConfig(); + config.setLoadManagerServiceUnitStateTableViewClassName( + ServiceUnitStateMetadataStoreTableViewImpl.class.getName()); + return config; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TableViewTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TableViewTest.java index 61ab4de8a3294..5448751160a9c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TableViewTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TableViewTest.java @@ -173,6 +173,9 @@ public void testRefreshAPI(int partition) throws Exception { TableView tv = pulsarClient.newTableView(Schema.BYTES) .topic(topic) .create(); + // Verify refresh can handle the case when the topic is empty + tv.refreshAsync().get(3, TimeUnit.SECONDS); + // 2. Add a listen action to provide the test environment. // The listen action will be triggered when there are incoming messages every time. // This is a sync operation, so sleep in the listen action can slow down the reading rate of messages. diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java index 4f52060497864..17b49828eeced 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.ReaderBuilder; @@ -259,7 +260,11 @@ private void handleMessage(Message msg) { @Override public CompletableFuture refreshAsync() { CompletableFuture completableFuture = new CompletableFuture<>(); - reader.thenCompose(reader -> getLastMessageIds(reader).thenAccept(lastMessageIds -> { + reader.thenCompose(reader -> getLastMessageIdOfNonEmptyTopics(reader).thenAccept(lastMessageIds -> { + if (lastMessageIds.isEmpty()) { + completableFuture.complete(null); + return; + } // After get the response of lastMessageIds, put the future and result into `refreshMap` // and then filter out partitions that has been read to the lastMessageID. pendingRefreshRequests.put(completableFuture, lastMessageIds); @@ -291,8 +296,12 @@ private CompletableFuture readAllExistingMessages(Reader reader) { AtomicLong messagesRead = new AtomicLong(); CompletableFuture future = new CompletableFuture<>(); - getLastMessageIds(reader).thenAccept(maxMessageIds -> { - readAllExistingMessages(reader, future, startTime, messagesRead, maxMessageIds); + getLastMessageIdOfNonEmptyTopics(reader).thenAccept(lastMessageIds -> { + if (lastMessageIds.isEmpty()) { + future.complete(null); + return; + } + readAllExistingMessages(reader, future, startTime, messagesRead, lastMessageIds); }).exceptionally(ex -> { future.completeExceptionally(ex); return null; @@ -300,13 +309,15 @@ private CompletableFuture readAllExistingMessages(Reader reader) { return future; } - private CompletableFuture> getLastMessageIds(Reader reader) { + private CompletableFuture> getLastMessageIdOfNonEmptyTopics(Reader reader) { return reader.getLastMessageIdsAsync().thenApply(lastMessageIds -> { - Map maxMessageIds = new ConcurrentHashMap<>(); + Map lastMessageIdMap = new ConcurrentHashMap<>(); lastMessageIds.forEach(topicMessageId -> { - maxMessageIds.put(topicMessageId.getOwnerTopic(), topicMessageId); + if (((MessageIdAdv) topicMessageId).getEntryId() >= 0) { + lastMessageIdMap.put(topicMessageId.getOwnerTopic(), topicMessageId); + } // else: a negative entry id represents an empty topic so that we don't have to read messages from it }); - return maxMessageIds; + return lastMessageIdMap; }); } From 9980967d777040706f15cc4a976af81d85c7faa6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 30 Sep 2024 17:10:15 +0300 Subject: [PATCH 021/327] [improve] Upgrade Pulsar Python client in docker image to 3.5.0 (#23377) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9f2330d7c75e1..881a1541c5eaf 100644 --- a/pom.xml +++ b/pom.xml @@ -82,7 +82,7 @@ flexible messaging model and an intuitive client API. ${maven.compiler.target} 8 - 3.4.0 + 3.5.0 21 From e0b754dd3938a2d142623001dbb15c92cc2f5cb4 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Mon, 30 Sep 2024 18:45:22 -0700 Subject: [PATCH 022/327] [improve][broker] check system topic is used before configuring system topic (ExtensibleLoadManagerImpl only) (#23381) --- .../extensions/ExtensibleLoadManagerImpl.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 841f9bfb669d4..d8a279b854576 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -50,6 +50,7 @@ import org.apache.pulsar.broker.loadbalance.LoadManager; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewSyncer; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; @@ -992,7 +993,7 @@ protected void monitor() { if (isChannelOwner) { // System topic config might fail due to the race condition // with topic policy init(Topic policies cache have not init). - if (!configuredSystemTopics) { + if (isPersistentSystemTopicUsed() && !configuredSystemTopics) { configuredSystemTopics = configureSystemTopics(pulsar, COMPACTION_THRESHOLD); } if (role != Leader) { @@ -1080,4 +1081,11 @@ boolean running() { private boolean disabled() { return state.get() == State.DISABLED; } + + private boolean isPersistentSystemTopicUsed() { + return ServiceUnitStateTableViewImpl.class.getName() + .equals(pulsar.getConfiguration().getLoadManagerServiceUnitStateTableViewClassName()); + } + + } From 5b98d371922832b78c596c33042932c660bea0c4 Mon Sep 17 00:00:00 2001 From: Philipp Dolif <52791955+phil-cd@users.noreply.github.com> Date: Tue, 1 Oct 2024 13:32:02 +0200 Subject: [PATCH 023/327] [feat] Use producer name and sequence number as fallback key in Key_Shared implementation (#23219) --- .../broker/service/EntryAndMetadata.java | 3 + .../client/api/KeySharedSubscriptionTest.java | 66 +++++++++++-------- .../pulsar/client/impl/ConsumerBase.java | 8 ++- .../pulsar/common/protocol/Commands.java | 3 + .../common/compression/CommandsTest.java | 24 +++++-- .../testclient/PerformanceProducerTest.java | 42 +++++------- .../integration/messaging/MessagingBase.java | 7 +- 7 files changed, 88 insertions(+), 65 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java index 70643d5de2a3f..efa89a8ff16f6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java @@ -55,6 +55,9 @@ public byte[] getStickyKey() { return metadata.getOrderingKey(); } else if (metadata.hasPartitionKey()) { return metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); + } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { + String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); + return fallbackKey.getBytes(StandardCharsets.UTF_8); } } return "NONE_KEY".getBytes(StandardCharsets.UTF_8); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index ddf7b0f1d5ee2..c08c37b413f4f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -44,6 +44,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Random; import java.util.Set; @@ -338,11 +339,11 @@ public void testConsumerCrashSendAndReceiveWithHashRangeAutoSplitStickyKeyConsum } @Test(dataProvider = "data") - public void testNonKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( + public void testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( String topicType, boolean enableBatch ) throws PulsarClientException { - String topic = topicType + "://public/default/key_shared_none_key-" + UUID.randomUUID(); + String topic = topicType + "://public/default/key_shared_no_key-" + UUID.randomUUID(); @Cleanup Consumer consumer1 = createConsumer(topic); @@ -362,13 +363,13 @@ public void testNonKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelec .send(); } - receive(Lists.newArrayList(consumer1, consumer2, consumer3)); + receiveAndCheckDistribution(Lists.newArrayList(consumer1, consumer2, consumer3), 100); } @Test(dataProvider = "batch") - public void testNonKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) + public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) throws PulsarClientException { - String topic = "persistent://public/default/key_shared_none_key_exclusive-" + UUID.randomUUID(); + String topic = "persistent://public/default/key_shared_no_key_exclusive-" + UUID.randomUUID(); @Cleanup Consumer consumer1 = createConsumer(topic, KeySharedPolicy.stickyHashRange() @@ -385,21 +386,32 @@ public void testNonKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelec @Cleanup Producer producer = createProducer(topic, enableBatch); + int consumer1ExpectMessages = 0; + int consumer2ExpectMessages = 0; + int consumer3ExpectMessages = 0; + for (int i = 0; i < 100; i++) { producer.newMessage() .value(i) .send(); + + String fallbackKey = producer.getProducerName() + "-" + producer.getLastSequenceId(); + int slot = Murmur3_32Hash.getInstance().makeHash(fallbackKey.getBytes()) + % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; + if (slot <= 20000) { + consumer1ExpectMessages++; + } else if (slot <= 40000) { + consumer2ExpectMessages++; + } else { + consumer3ExpectMessages++; + } } - int slot = Murmur3_32Hash.getInstance().makeHash("NONE_KEY".getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; + List, Integer>> checkList = new ArrayList<>(); - if (slot <= 20000) { - checkList.add(new KeyValue<>(consumer1, 100)); - } else if (slot <= 40000) { - checkList.add(new KeyValue<>(consumer2, 100)); - } else { - checkList.add(new KeyValue<>(consumer3, 100)); - } + checkList.add(new KeyValue<>(consumer1, consumer1ExpectMessages)); + checkList.add(new KeyValue<>(consumer2, consumer2ExpectMessages)); + checkList.add(new KeyValue<>(consumer3, consumer3ExpectMessages)); + receiveAndCheck(checkList); } @@ -1740,19 +1752,17 @@ private void receiveAndCheckDistribution(List> consumers, int expect private void receiveAndCheck(List, Integer>> checkList) throws PulsarClientException { Map> consumerKeys = new HashMap<>(); for (KeyValue, Integer> check : checkList) { - if (check.getValue() % 2 != 0) { - throw new IllegalArgumentException(); - } + Consumer consumer = check.getKey(); int received = 0; Map> lastMessageForKey = new HashMap<>(); for (Integer i = 0; i < check.getValue(); i++) { - Message message = check.getKey().receive(); + Message message = consumer.receive(); if (i % 2 == 0) { - check.getKey().acknowledge(message); + consumer.acknowledge(message); } String key = message.hasOrderingKey() ? new String(message.getOrderingKey()) : message.getKey(); log.info("[{}] Receive message key: {} value: {} messageId: {}", - check.getKey().getConsumerName(), key, message.getValue(), message.getMessageId()); + consumer.getConsumerName(), key, message.getValue(), message.getMessageId()); // check messages is order by key if (lastMessageForKey.get(key) == null) { Assert.assertNotNull(message); @@ -1761,8 +1771,8 @@ private void receiveAndCheck(List, Integer>> checkLis .compareTo(lastMessageForKey.get(key).getValue()) > 0); } lastMessageForKey.put(key, message); - consumerKeys.putIfAbsent(check.getKey(), new HashSet<>()); - consumerKeys.get(check.getKey()).add(key); + consumerKeys.putIfAbsent(consumer, new HashSet<>()); + consumerKeys.get(consumer).add(key); received++; } Assert.assertEquals(check.getValue().intValue(), received); @@ -1771,12 +1781,12 @@ private void receiveAndCheck(List, Integer>> checkLis // messages not acked, test redelivery lastMessageForKey = new HashMap<>(); for (int i = 0; i < redeliveryCount; i++) { - Message message = check.getKey().receive(); + Message message = consumer.receive(); received++; - check.getKey().acknowledge(message); + consumer.acknowledge(message); String key = message.hasOrderingKey() ? new String(message.getOrderingKey()) : message.getKey(); log.info("[{}] Receive redeliver message key: {} value: {} messageId: {}", - check.getKey().getConsumerName(), key, message.getValue(), message.getMessageId()); + consumer.getConsumerName(), key, message.getValue(), message.getMessageId()); // check redelivery messages is order by key if (lastMessageForKey.get(key) == null) { Assert.assertNotNull(message); @@ -1788,16 +1798,16 @@ private void receiveAndCheck(List, Integer>> checkLis } Message noMessages = null; try { - noMessages = check.getKey().receive(100, TimeUnit.MILLISECONDS); + noMessages = consumer.receive(100, TimeUnit.MILLISECONDS); } catch (PulsarClientException ignore) { } Assert.assertNull(noMessages, "redeliver too many messages."); Assert.assertEquals((check.getValue() + redeliveryCount), received); } Set allKeys = new HashSet<>(); - consumerKeys.forEach((k, v) -> v.forEach(key -> { + consumerKeys.forEach((k, v) -> v.stream().filter(Objects::nonNull).forEach(key -> { assertTrue(allKeys.add(key), - "Key "+ key + "is distributed to multiple consumers." ); + "Key " + key + " is distributed to multiple consumers." ); })); } 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 111cbdb8a8ef3..3073f3a833487 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 @@ -1192,11 +1192,13 @@ protected void callMessageListener(Message msg) { static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); protected byte[] peekMessageKey(Message msg) { byte[] key = NONE_KEY; - if (msg.hasKey()) { - key = msg.getKeyBytes(); - } if (msg.hasOrderingKey()) { key = msg.getOrderingKey(); + } else if (msg.hasKey()) { + key = msg.getKeyBytes(); + } else if (msg.getProducerName() != null) { + String fallbackKey = msg.getProducerName() + "-" + msg.getSequenceId(); + key = fallbackKey.getBytes(StandardCharsets.UTF_8); } return key; } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 3fb2fd5ad3d25..15b5676094ec1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -1983,6 +1983,9 @@ public static byte[] peekStickyKey(ByteBuf metadataAndPayload, String topic, Str return Base64.getDecoder().decode(metadata.getPartitionKey()); } return metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); + } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { + String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); + return fallbackKey.getBytes(StandardCharsets.UTF_8); } } catch (Throwable t) { log.error("[{}] [{}] Failed to peek sticky key from the message metadata", topic, subscription, t); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/compression/CommandsTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/compression/CommandsTest.java index 42f1a58100283..a1f79b7ae7faf 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/compression/CommandsTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/compression/CommandsTest.java @@ -98,9 +98,11 @@ private int computeChecksum(MessageMetadata msgMetadata, ByteBuf compressedPaylo public void testPeekStickyKey() { String message = "msg-1"; String partitionedKey = "key1"; + String producerName = "testProducer"; + int sequenceId = 1; MessageMetadata messageMetadata2 = new MessageMetadata() - .setSequenceId(1) - .setProducerName("testProducer") + .setSequenceId(sequenceId) + .setProducerName(producerName) .setPartitionKey(partitionedKey) .setPartitionKeyB64Encoded(false) .setPublishTime(System.currentTimeMillis()); @@ -113,16 +115,28 @@ public void testPeekStickyKey() { // test 64 encoded String partitionedKey2 = Base64.getEncoder().encodeToString("key2".getBytes(UTF_8)); MessageMetadata messageMetadata = new MessageMetadata() - .setSequenceId(1) - .setProducerName("testProducer") + .setSequenceId(sequenceId) + .setProducerName(producerName) .setPartitionKey(partitionedKey2) .setPartitionKeyB64Encoded(true) .setPublishTime(System.currentTimeMillis()); ByteBuf byteBuf2 = serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, Unpooled.copiedBuffer(message.getBytes(UTF_8))); byte[] bytes2 = Commands.peekStickyKey(byteBuf2, "topic-2", "sub-2"); - String key2 = Base64.getEncoder().encodeToString(bytes2);; + String key2 = Base64.getEncoder().encodeToString(bytes2); Assert.assertEquals(partitionedKey2, key2); ReferenceCountUtil.safeRelease(byteBuf2); + // test fallback key if no key given in message metadata + String fallbackPartitionedKey = producerName + "-" + sequenceId; + MessageMetadata messageMetadataWithoutKey = new MessageMetadata() + .setSequenceId(sequenceId) + .setProducerName(producerName) + .setPublishTime(System.currentTimeMillis()); + ByteBuf byteBuf3 = serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadataWithoutKey, + Unpooled.copiedBuffer(message.getBytes(UTF_8))); + byte[] bytes3 = Commands.peekStickyKey(byteBuf3, "topic-3", "sub-3"); + String key3 = new String(bytes3); + Assert.assertEquals(fallbackPartitionedKey, key3); + ReferenceCountUtil.safeRelease(byteBuf3); } } diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java index d0b25c6971697..519bed6cdb5ae 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java @@ -98,26 +98,20 @@ public void testMsgKey() throws Exception { thread.start(); - int count1 = 0; - int count2 = 0; - for (int i = 0; i < 10; i++) { - Message message = consumer1.receive(1, TimeUnit.SECONDS); - if (message == null) { - break; - } - count1++; - consumer1.acknowledge(message); - } - for (int i = 0; i < 10; i++) { - Message message = consumer2.receive(1, TimeUnit.SECONDS); - if (message == null) { - break; - } - count2++; - consumer2.acknowledge(message); - } - //in key_share mode, only one consumer can get msg - Assert.assertTrue(count1 == 0 || count2 == 0); + // in key_shared mode if no message key is set, both consumers should receive messages + Awaitility.await() + .untilAsserted(() -> { + Message message = consumer1.receive(1, TimeUnit.SECONDS); + assertNotNull(message); + consumer1.acknowledge(message); + }); + + Awaitility.await() + .untilAsserted(() -> { + Message message = consumer2.receive(1, TimeUnit.SECONDS); + assertNotNull(message); + consumer2.acknowledge(message); + }); consumer1.close(); consumer2.close(); @@ -149,19 +143,15 @@ public void testMsgKey() throws Exception { Awaitility.await() .untilAsserted(() -> { Message message = newConsumer1.receive(1, TimeUnit.SECONDS); - if (message != null) { - newConsumer1.acknowledge(message); - } assertNotNull(message); + newConsumer1.acknowledge(message); }); Awaitility.await() .untilAsserted(() -> { Message message = newConsumer2.receive(1, TimeUnit.SECONDS); - if (message != null) { - newConsumer2.acknowledge(message); - } assertNotNull(message); + newConsumer2.acknowledge(message); }); thread2.interrupt(); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/MessagingBase.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/MessagingBase.java index 0e7106ef65ea1..ddedacc531a7c 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/MessagingBase.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/MessagingBase.java @@ -36,6 +36,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -150,11 +151,11 @@ protected String getPartitionedTopic(String topicPrefix, boolean isPersistent, i } } } - // Make sure key will not be distributed to multiple consumers + // Make sure key will not be distributed to multiple consumers (except null key) Set allKeys = Sets.newHashSet(); - consumerKeys.forEach((k, v) -> v.forEach(key -> { + consumerKeys.forEach((k, v) -> v.stream().filter(Objects::nonNull).forEach(key -> { assertTrue(allKeys.add(key), - "Key "+ key + "is distributed to multiple consumers" ); + "Key " + key + " is distributed to multiple consumers" ); })); assertEquals(messagesReceived.size(), messagesToReceive); } From 9eeffe595b6c2312b1b92eb8b9606639f25ab276 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 1 Oct 2024 12:53:45 -0700 Subject: [PATCH 024/327] [fix][broker] Support large number of unack message store for cursor recovery (#9292) --- .../mledger/ManagedLedgerConfig.java | 4 +- .../mledger/impl/ManagedCursorImpl.java | 64 +++++++++++++++++-- .../mledger/impl/RangeSetWrapper.java | 29 +++++++++ .../src/main/proto/MLDataFormats.proto | 6 ++ .../mledger/impl/ManagedCursorTest.java | 11 ++-- .../mledger/impl/ManagedLedgerBkTest.java | 47 +++++++++++++- .../BrokerRegistryIntegrationTest.java | 2 +- .../ConcurrentOpenLongPairRangeSet.java | 41 ++++++++++++ .../util/collections/LongPairRangeSet.java | 14 ++++ .../collections/OpenLongPairRangeSet.java | 46 +++++++++++++ 10 files changed, 249 insertions(+), 15 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 03439f93ccad8..a24251450b4f4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -505,8 +505,10 @@ public int getMaxUnackedRangesToPersistInMetadataStore() { return maxUnackedRangesToPersistInMetadataStore; } - public void setMaxUnackedRangesToPersistInMetadataStore(int maxUnackedRangesToPersistInMetadataStore) { + public ManagedLedgerConfig setMaxUnackedRangesToPersistInMetadataStore( + int maxUnackedRangesToPersistInMetadataStore) { this.maxUnackedRangesToPersistInMetadataStore = maxUnackedRangesToPersistInMetadataStore; + return this; } /** diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index e27814eadd0b5..b39fd231cdc06 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -59,6 +59,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import java.util.function.Predicate; +import java.util.stream.Collectors; import java.util.stream.LongStream; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; @@ -91,12 +92,15 @@ import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongListMap; import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo.Builder; import org.apache.bookkeeper.mledger.proto.MLDataFormats.StringProperty; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.apache.pulsar.common.util.DateFormatter; @@ -606,9 +610,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } Position position = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); - if (positionInfo.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); - } + recoverIndividualDeletedMessages(positionInfo); if (getConfig().isDeletionAtBatchIndexLevelEnabled() && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); @@ -627,6 +629,45 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } } + public void recoverIndividualDeletedMessages(PositionInfo positionInfo) { + if (positionInfo.getIndividualDeletedMessagesCount() > 0) { + recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); + } else if (positionInfo.getIndividualDeletedMessageRangesCount() > 0) { + List rangeList = positionInfo.getIndividualDeletedMessageRangesList(); + try { + Map rangeMap = rangeList.stream().collect(Collectors.toMap(LongListMap::getKey, + list -> list.getValuesList().stream().mapToLong(i -> i).toArray())); + individualDeletedMessages.build(rangeMap); + } catch (Exception e) { + log.warn("[{}]-{} Failed to recover individualDeletedMessages from serialized data", ledger.getName(), + name, e); + } + } + } + + private List buildLongPropertiesMap(Map properties) { + if (properties.isEmpty()) { + return Collections.emptyList(); + } + List longListMap = new ArrayList<>(); + MutableInt serializedSize = new MutableInt(); + properties.forEach((id, ranges) -> { + if (ranges == null || ranges.length <= 0) { + return; + } + org.apache.bookkeeper.mledger.proto.MLDataFormats.LongListMap.Builder lmBuilder = LongListMap.newBuilder() + .setKey(id); + for (long range : ranges) { + lmBuilder.addValues(range); + } + LongListMap lm = lmBuilder.build(); + longListMap.add(lm); + serializedSize.add(lm.getSerializedSize()); + }); + individualDeletedMessagesSerializedSize = serializedSize.toInteger(); + return longListMap; + } + private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { lock.writeLock().lock(); try { @@ -3125,12 +3166,23 @@ private List buildBatchEntryDeletio void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { Position position = mdEntry.newPosition; - PositionInfo pi = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) + Builder piBuilder = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) .setEntryId(position.getEntryId()) - .addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()) .addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()) - .addAllProperties(buildPropertiesMap(mdEntry.properties)).build(); + .addAllProperties(buildPropertiesMap(mdEntry.properties)); + Map internalRanges = null; + try { + internalRanges = individualDeletedMessages.toRanges(getConfig().getMaxUnackedRangesToPersist()); + } catch (Exception e) { + log.warn("[{}]-{} Failed to serialize individualDeletedMessages", ledger.getName(), name, e); + } + if (internalRanges != null && !internalRanges.isEmpty()) { + piBuilder.addAllIndividualDeletedMessageRanges(buildLongPropertiesMap(internalRanges)); + } else { + piBuilder.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); + } + PositionInfo pi = piBuilder.build(); if (log.isDebugEnabled()) { log.debug("[{}] Cursor {} Appending to ledger={} position={}", ledger.getName(), name, lh.getId(), diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index a55e6444b2fd9..11cce409bec54 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; @@ -142,6 +143,16 @@ public Range lastRange() { return rangeSet.lastRange(); } + @Override + public Map toRanges(int maxRanges) { + return rangeSet.toRanges(maxRanges); + } + + @Override + public void build(Map internalRange) { + rangeSet.build(internalRange); + } + @Override public int cardinality(long lowerKey, long lowerValue, long upperKey, long upperValue) { return rangeSet.cardinality(lowerKey, lowerValue, upperKey, upperValue); @@ -176,4 +187,22 @@ public boolean isDirtyLedgers(long ledgerId) { public String toString() { return rangeSet.toString(); } + + @Override + public int hashCode() { + return rangeSet.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RangeSetWrapper)) { + return false; + } + if (this == obj) { + return true; + } + @SuppressWarnings("rawtypes") + RangeSetWrapper set = (RangeSetWrapper) obj; + return this.rangeSet.equals(set.rangeSet); + } } diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index fdffed6762db7..f196649df0fdf 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -82,6 +82,7 @@ message PositionInfo { // Store which index in the batch message has been deleted repeated BatchedEntryDeletionIndexInfo batchedEntryDeletionIndexInfo = 5; + repeated LongListMap individualDeletedMessageRanges = 6; } message NestedPositionInfo { @@ -89,6 +90,11 @@ message NestedPositionInfo { required int64 entryId = 2; } +message LongListMap { + required int64 key = 1; + repeated int64 values = 2; +} + message MessageRange { required NestedPositionInfo lowerEndpoint = 1; required NestedPositionInfo upperEndpoint = 2; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 8913c4013b4ab..1067cda441f6a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -3223,7 +3223,7 @@ public void testOutOfOrderDeletePersistenceIntoLedgerWithClose() throws Exceptio managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore(10); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerName, managedLedgerConfig); - ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName); + final ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName); List addedPositions = new ArrayList<>(); for (int i = 0; i < totalAddEntries; i++) { @@ -3269,7 +3269,8 @@ public void operationFailed(MetaStoreException e) { LedgerEntry entry = seq.nextElement(); PositionInfo positionInfo; positionInfo = PositionInfo.parseFrom(entry.getEntry()); - individualDeletedMessagesCount.set(positionInfo.getIndividualDeletedMessagesCount()); + c1.recoverIndividualDeletedMessages(positionInfo); + individualDeletedMessagesCount.set(c1.getIndividuallyDeletedMessagesSet().asRanges().size()); } catch (Exception e) { } latch.countDown(); @@ -3286,12 +3287,12 @@ public void operationFailed(MetaStoreException e) { @Cleanup("shutdown") ManagedLedgerFactory factory2 = new ManagedLedgerFactoryImpl(metadataStore, bkc); ledger = (ManagedLedgerImpl) factory2.open(ledgerName, managedLedgerConfig); - c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + ManagedCursorImpl reopenCursor = (ManagedCursorImpl) ledger.openCursor("c1"); // verify cursor has been recovered - assertEquals(c1.getNumberOfEntriesInBacklog(false), totalAddEntries / 2); + assertEquals(reopenCursor.getNumberOfEntriesInBacklog(false), totalAddEntries / 2); // try to read entries which should only read non-deleted positions - List entries = c1.readEntries(totalAddEntries); + List entries = reopenCursor.readEntries(totalAddEntries); assertEquals(entries.size(), totalAddEntries / 2); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java index cd1dcf05c3708..9635376a782d3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java @@ -23,7 +23,6 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -import io.netty.buffer.ByteBuf; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -34,7 +33,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import lombok.Cleanup; + import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperTestClient; import org.apache.bookkeeper.client.api.DigestType; @@ -53,9 +52,13 @@ import org.apache.bookkeeper.mledger.util.ThrowableToStringUtil; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; +import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.awaitility.Awaitility; import org.testng.annotations.Test; +import io.netty.buffer.ByteBuf; +import lombok.Cleanup; + public class ManagedLedgerBkTest extends BookKeeperClusterTestCase { public ManagedLedgerBkTest() { @@ -587,4 +590,44 @@ public void testPeriodicRollover() throws Exception { Awaitility.await().until(() -> cursorImpl.getCursorLedger() != currentLedgerId); } + /** + * This test validates that cursor serializes and deserializes individual-ack list from the bk-ledger. + * + * @throws Exception + */ + @Test + public void testUnackmessagesAndRecovery() throws Exception { + ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig(); + factoryConf.setMaxCacheSize(0); + + ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf); + + ManagedLedgerConfig config = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) + .setAckQuorumSize(1).setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1) + .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1); + ManagedLedger ledger = factory.open("my_test_unack_messages", config); + ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + + int totalEntries = 100; + for (int i = 0; i < totalEntries; i++) { + Position p = ledger.addEntry("entry".getBytes()); + if (i % 2 == 0) { + cursor.delete(p); + } + } + + LongPairRangeSet unackMessagesBefore = cursor.getIndividuallyDeletedMessagesSet(); + + ledger.close(); + + // open and recover cursor + ledger = factory.open("my_test_unack_messages", config); + cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + + LongPairRangeSet unackMessagesAfter = cursor.getIndividuallyDeletedMessagesSet(); + assertTrue(unackMessagesBefore.equals(unackMessagesAfter)); + + ledger.close(); + factory.shutdown(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java index 232088afb94fe..e975671fa12e8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -80,7 +80,7 @@ public void testRecoverFromNodeDeletion() throws Exception { Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); pulsar.getLocalMetadataStore().delete(brokerMetadataPath, Optional.empty()); - Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( + Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> Assert.assertEquals( brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); // If the node is deleted by unregister(), it should not recreate the path diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java index 6e45401978546..51f4a9ac51c90 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java @@ -18,16 +18,21 @@ */ package org.apache.pulsar.common.util.collections; +import static java.util.BitSet.valueOf; import static java.util.Objects.requireNonNull; import com.google.common.collect.BoundType; import com.google.common.collect.Range; import java.util.ArrayList; import java.util.BitSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.NavigableMap; +import java.util.Objects; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.lang.mutable.MutableInt; /** @@ -253,6 +258,42 @@ public Range lastRange() { return Range.openClosed(consumer.apply(lastSet.getKey(), lower), consumer.apply(lastSet.getKey(), upper)); } + @Override + public Map toRanges(int maxRanges) { + Map internalBitSetMap = new HashMap<>(); + AtomicInteger rangeCount = new AtomicInteger(); + rangeBitSetMap.forEach((id, bmap) -> { + if (rangeCount.getAndAdd(bmap.cardinality()) > maxRanges) { + return; + } + internalBitSetMap.put(id, bmap.toLongArray()); + }); + return internalBitSetMap; + } + + @Override + public void build(Map internalRange) { + internalRange.forEach((id, ranges) -> rangeBitSetMap.put(id, valueOf(ranges))); + } + + @Override + public int hashCode() { + return Objects.hashCode(rangeBitSetMap); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof ConcurrentOpenLongPairRangeSet)) { + return false; + } + if (this == obj) { + return true; + } + @SuppressWarnings("rawtypes") + ConcurrentOpenLongPairRangeSet set = (ConcurrentOpenLongPairRangeSet) obj; + return this.rangeBitSetMap.equals(set.rangeBitSetMap); + } + @Override public int cardinality(long lowerKey, long lowerValue, long upperKey, long upperValue) { NavigableMap subMap = rangeBitSetMap.subMap(lowerKey, true, upperKey, true); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java index 8aad5587dfd38..df74857245bb3 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; import java.util.Set; import lombok.EqualsAndHashCode; @@ -136,6 +137,19 @@ public interface LongPairRangeSet> { */ Range lastRange(); + default Map toRanges(int maxRanges) { + throw new UnsupportedOperationException(); + } + + /** + * Build {@link LongPairRangeSet} using internal ranges returned by {@link #toRanges(int)} . + * + * @param ranges + */ + default void build(Map ranges) { + throw new UnsupportedOperationException(); + } + /** * Return the number bit sets to true from lower (inclusive) to upper (inclusive). */ diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java index 6df6d414871ec..3076c6c5c5fa1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java @@ -18,16 +18,21 @@ */ package org.apache.pulsar.common.util.collections; +import static java.util.BitSet.valueOf; import static java.util.Objects.requireNonNull; import com.google.common.collect.BoundType; import com.google.common.collect.Range; import java.util.ArrayList; import java.util.BitSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.NavigableMap; +import java.util.Objects; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.lang.mutable.MutableInt; @@ -250,6 +255,47 @@ public Range lastRange() { return Range.openClosed(consumer.apply(lastSet.getKey(), lower), consumer.apply(lastSet.getKey(), upper)); } + @Override + public Map toRanges(int maxRanges) { + Map internalBitSetMap = new HashMap<>(); + AtomicInteger rangeCount = new AtomicInteger(); + rangeBitSetMap.forEach((id, bmap) -> { + if (rangeCount.getAndAdd(bmap.cardinality()) > maxRanges) { + return; + } + internalBitSetMap.put(id, bmap.toLongArray()); + }); + return internalBitSetMap; + } + + @Override + public void build(Map internalRange) { + internalRange.forEach((id, ranges) -> { + BitSet bitset = createNewBitSet(); + bitset.or(valueOf(ranges)); + rangeBitSetMap.put(id, bitset); + }); + } + + + @Override + public int hashCode() { + return Objects.hashCode(rangeBitSetMap); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof OpenLongPairRangeSet)) { + return false; + } + if (this == obj) { + return true; + } + @SuppressWarnings("rawtypes") + OpenLongPairRangeSet set = (OpenLongPairRangeSet) obj; + return this.rangeBitSetMap.equals(set.rangeBitSetMap); + } + @Override public int cardinality(long lowerKey, long lowerValue, long upperKey, long upperValue) { NavigableMap subMap = rangeBitSetMap.subMap(lowerKey, true, upperKey, true); From d2c91b1e1a8fc2fb233eb2856ddb6f53511ba201 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 1 Oct 2024 22:54:32 +0300 Subject: [PATCH 025/327] [fix][broker] Cancel possible pending replay read in cancelPendingRead (#23384) --- .../persistent/PersistentDispatcherMultipleConsumers.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 73ad2cf0a3dee..d479d8f384ee9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -650,8 +650,9 @@ public synchronized CompletableFuture disconnectAllConsumers( @Override protected void cancelPendingRead() { - if (havePendingRead && cursor.cancelPendingReadRequest()) { + if ((havePendingRead || havePendingReplayRead) && cursor.cancelPendingReadRequest()) { havePendingRead = false; + havePendingReplayRead = false; } } From 50802bea7288f34b39ee19a47ed31b3629a9ddda Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Tue, 1 Oct 2024 15:37:43 -0700 Subject: [PATCH 026/327] [improve][pip] PIP-381: Handle large PositionInfo state (#23328) --- pip/pip-381-large-positioninfo.md | 153 ++++++++++++++++++++++++++++++ 1 file changed, 153 insertions(+) create mode 100644 pip/pip-381-large-positioninfo.md diff --git a/pip/pip-381-large-positioninfo.md b/pip/pip-381-large-positioninfo.md new file mode 100644 index 0000000000000..9dbe1cc7935e3 --- /dev/null +++ b/pip/pip-381-large-positioninfo.md @@ -0,0 +1,153 @@ +# PIP-381: Handle large PositionInfo state + +# Background knowledge + +In case of KEY_SHARED subscription and out-of-order acknowledgments, +the PositionInfo state can be persisted to preserve the state, +with configurable maximum number of ranges to persist: + +``` +# Max number of "acknowledgment holes" that are going to be persistently stored. +# When acknowledging out of order, a consumer will leave holes that are supposed +# to be quickly filled by acking all the messages. The information of which +# messages are acknowledged is persisted by compressing in "ranges" of messages +# that were acknowledged. After the max number of ranges is reached, the information +# will only be tracked in memory and messages will be redelivered in case of +# crashes. +managedLedgerMaxUnackedRangesToPersist=10000 +``` + +The PositionInfo state is stored to the BookKeeper as a single entry, and it can grow large if the number of ranges is large. +Currently, this means that BookKeeper can fail persisting too large PositionInfo state, e.g. over 1MB +by default and the ManagedCursor recovery on topic reload might not succeed. + +There is an abandoned PIP-81 for similar problem, this PIP takes over. + +# Motivation + +While keeping the number of ranges low to prevent such problems is a common sense solution, there are cases +where the higher number of ranges is required. For example, in case of the JMS protocol handler, +JMS consumers with filters may end up processing data out of order and/or at different speed, +and the number of ranges can grow large. + +# Goals + +Store the PositionInfo state in a BookKeeper ledger as multiple entries if the state grows too large to be stored as a single entry. + +## In Scope + +Transparent backwards compatibility if the PositionInfo state is small enough. + +## Out of Scope + +Backwards compatibility in case of the PositionInfo state is too large to be stored as a single entry. + +# High Level Design + +Cursor state writes and reads are happening at the same cases as currently, without changes. + +Write path: + +1. serialize the PositionInfo state to a byte array. +2. if the byte array is smaller than the threshold, store it as a single entry, as now. Done. +3. if the byte array is larger than the threshold, split it to smaller chunks and store the chunks in a BookKeeper ledger. +4. write the "footer" into the metadata store as a last entry. + +See `persistPositionToLedger()` in `ManagedCursorImpl` for the implementation. + +The footer is a JSON representation of + +```java + public static final class ChunkSequenceFooter { + private int numParts; + private int length; + } +``` + +Read path: + +1. read the last entry from the metadata store. +2. if the entry does not appear to be a JSON, treat it as serialized PositionInfo state and use it as is. Done. +3. if the footer is a JSON, parse number of chunks and length from the json. +4. read the chunks from the BookKeeper ledger (entries from `startPos = footerPosition - chunkSequenceFooter.numParts` to `footerPosition - 1`) and merge them. +5. parse the merged byte array as a PositionInfo state. + +See `recoverFromLedgerByEntryId()` in `ManagedCursorImpl` for the implementation. + +## Design & Implementation Details + +Proposed implementation: https://github.com/apache/pulsar/pull/22799 + +## Public-facing Changes + +Nothing + +### Public API + +None + +### Binary protocol + +No public-facing changes + +### Configuration + +* **managedLedgerMaxUnackedRangesToPersist**: int, default 10000 (existing parameter). Controls number of unacked ranges to store. +* **persistentUnackedRangesWithMultipleEntriesEnabled**: boolean, default false. If true, the PositionInfo state is stored as multiple entries in BookKeeper if it grows too large. +* **persistentUnackedRangesMaxEntrySize**: int, default 1MB. Maximum size of a single entry in BookKeeper, in bytes. +* **cursorInfoCompressionType**: string, default "NONE". Compression type to use for the PositionInfo state. + +### CLI + +None + +### Metrics + + + + +# Monitoring + +Existing monitoring should be sufficient. + +# Security Considerations + +N/A + +# Backward & Forward Compatibility + +## Upgrade + +Not affected, just upgrade. + +## Downgrade / Rollback + +Not affected, just downgrade **as long as the managedLedgerMaxUnackedRangesToPersist was in the range to fit it into a single entry in BK**. + +## Pulsar Geo-Replication Upgrade & Downgrade/Rollback Considerations + +Not affected AFAIK. + +# Alternatives + +1. Do nothing. Keep the number of ranges low. This does not fit some use cases. +2. Come up with an extremely efficient storage format for the unacked ranges to fit them into a single entry all the time for e.g. 10mil ranges. This breaks backwards compatibility and the feasibility is unclear. + +# General Notes + +# Links + +* Proposed implementation: https://github.com/apache/pulsar/pull/22799 +* PIP-81: https://github.com/apache/pulsar/wiki/PIP-81:-Split-the-individual-acknowledgments-into-multiple-entries +* PR that implements better storage format for the unacked ranges (alternative 2): https://github.com/apache/pulsar/pull/9292 + +ML discussion and voting threads: + +* Mailing List discussion thread: https://lists.apache.org/thread/8sm0h804v5914zowghrqxr92fp7c255d +* Mailing List voting thread: https://lists.apache.org/thread/q31fx0rox9tdt34xsmo1ol1l76q8vk99 From 1dad0788c0e3bdb2a2d76ba2908ddb30441460c3 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 1 Oct 2024 17:03:52 -0700 Subject: [PATCH 027/327] [improve][pip] Improve PIP process and voting timeline (#23387) --- pip/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pip/README.md b/pip/README.md index f386647e8c5c2..216cdd56298c6 100644 --- a/pip/README.md +++ b/pip/README.md @@ -77,7 +77,7 @@ The process works in the following way: sending a message using subject `[VOTE] PIP-xxx: {PIP TITLE}`. Make sure to include a link to the PIP PR in the body of the message. Make sure to update the PIP with a link to the vote. You can obtain it from [Apache Pony Mail](https://lists.apache.org/list.html?dev@pulsar.apache.org). Everyone is welcome to vote on the proposal, though only the vote of the PMC members will be considered binding. - It is required to have a lazy majority of at least 3 binding +1s votes. + The requirement is to have at least one binding +1 vote from a lazy majority if no binding -1 votes have been cast on the PIP. The vote should stay open for at least 48 hours. 9. When the vote is closed, if the outcome is positive, ask a PMC member (using voting thread on mailing list) to merge the PR. 10. If the outcome is negative, please close the PR (with a small comment that the close is a result of a vote). From c41c7e944d9a556dc02710314310457df82da502 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 2 Oct 2024 06:46:52 +0300 Subject: [PATCH 028/327] [improve] Configure Rocksdb to use musl libc flavor of the native library (#23375) --- docker/pulsar/Dockerfile | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index f3b0f3d944bdc..f8c22dc14a821 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -141,6 +141,8 @@ COPY --from=pulsar /pulsar /pulsar WORKDIR /pulsar ENV PATH=$PATH:$JAVA_HOME/bin:/pulsar/bin +# Use musl libc library for RocksDB +ENV ROCKSDB_MUSL_LIBC=true # The UID must be non-zero. Otherwise, it is arbitrary. No logic should rely on its specific value. ARG DEFAULT_USERNAME=pulsar From adb9014dbac21afdfb5fc252ac38e07ed2d6b19c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 2 Oct 2024 11:13:36 +0300 Subject: [PATCH 029/327] [fix][broker] Fix out-of-order issues with ConsistentHashingStickyKeyConsumerSelector (#23327) --- ...stentHashingStickyKeyConsumerSelector.java | 104 ++--- .../service/ConsumerIdentityWrapper.java | 70 ++++ .../service/ConsumerNameIndexTracker.java | 136 +++++++ ...tHashingStickyKeyConsumerSelectorTest.java | 366 +++++++++++++++++- .../service/ConsumerIdentityWrapperTest.java | 68 ++++ .../service/ConsumerNameIndexTrackerTest.java | 157 ++++++++ ...ckyKeyDispatcherMultipleConsumersTest.java | 9 +- .../org/apache/pulsar/client/api/Range.java | 11 +- 8 files changed, 853 insertions(+), 68 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapper.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerNameIndexTracker.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapperTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerNameIndexTrackerTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java index b2b2b512c8cfc..1ae9a6ff96b7d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java @@ -18,10 +18,8 @@ */ package org.apache.pulsar.broker.service; -import com.google.common.collect.Lists; import java.util.ArrayList; -import java.util.Comparator; -import java.util.LinkedHashMap; +import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -44,7 +42,9 @@ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyCons private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); // Consistent-Hash ring - private final NavigableMap> hashRing; + private final NavigableMap hashRing; + // Tracks the used consumer name indexes for each consumer name + private final ConsumerNameIndexTracker consumerNameIndexTracker = new ConsumerNameIndexTracker(); private final int numberOfPoints; @@ -57,21 +57,20 @@ public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints) { public CompletableFuture addConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { + ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); // Insert multiple points on the hash ring for every consumer // The points are deterministically added based on the hash of the consumer name for (int i = 0; i < numberOfPoints; i++) { - int hash = calculateHashForConsumerAndIndex(consumer, i); - hashRing.compute(hash, (k, v) -> { - if (v == null) { - return Lists.newArrayList(consumer); - } else { - if (!v.contains(consumer)) { - v.add(consumer); - v.sort(Comparator.comparing(Consumer::consumerName, String::compareTo)); - } - return v; - } - }); + int consumerNameIndex = + consumerNameIndexTracker.increaseConsumerRefCountAndReturnIndex(consumerIdentityWrapper); + int hash = calculateHashForConsumerAndIndex(consumer, consumerNameIndex, i); + // When there's a collision, the new consumer will replace the old one. + // This is a rare case, and it is acceptable to replace the old consumer since there + // are multiple points for each consumer. This won't affect the overall distribution significantly. + ConsumerIdentityWrapper removed = hashRing.put(hash, consumerIdentityWrapper); + if (removed != null) { + consumerNameIndexTracker.decreaseConsumerRefCount(removed); + } } return CompletableFuture.completedFuture(null); } finally { @@ -79,8 +78,19 @@ public CompletableFuture addConsumer(Consumer consumer) { } } - private static int calculateHashForConsumerAndIndex(Consumer consumer, int index) { - String key = consumer.consumerName() + KEY_SEPARATOR + index; + /** + * Calculate the hash for a consumer and hash ring point. + * The hash is calculated based on the consumer name, consumer name index, and hash ring point index. + * The resulting hash is used as the key to insert the consumer into the hash ring. + * + * @param consumer the consumer + * @param consumerNameIndex the index of the consumer name + * @param hashRingPointIndex the index of the hash ring point + * @return the hash value + */ + private static int calculateHashForConsumerAndIndex(Consumer consumer, int consumerNameIndex, + int hashRingPointIndex) { + String key = consumer.consumerName() + KEY_SEPARATOR + consumerNameIndex + KEY_SEPARATOR + hashRingPointIndex; return Murmur3_32Hash.getInstance().makeHash(key.getBytes()); } @@ -88,20 +98,16 @@ private static int calculateHashForConsumerAndIndex(Consumer consumer, int index public void removeConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { - // Remove all the points that were added for this consumer - for (int i = 0; i < numberOfPoints; i++) { - int hash = calculateHashForConsumerAndIndex(consumer, i); - hashRing.compute(hash, (k, v) -> { - if (v == null) { - return null; - } else { - v.removeIf(c -> c.equals(consumer)); - if (v.isEmpty()) { - v = null; - } - return v; + ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); + int consumerNameIndex = consumerNameIndexTracker.getTrackedIndex(consumerIdentityWrapper); + if (consumerNameIndex > -1) { + // Remove all the points that were added for this consumer + for (int i = 0; i < numberOfPoints; i++) { + int hash = calculateHashForConsumerAndIndex(consumer, consumerNameIndex, i); + if (hashRing.remove(hash, consumerIdentityWrapper)) { + consumerNameIndexTracker.decreaseConsumerRefCount(consumerIdentityWrapper); } - }); + } } } finally { rwLock.writeLock().unlock(); @@ -115,16 +121,13 @@ public Consumer select(int hash) { if (hashRing.isEmpty()) { return null; } - - List consumerList; - Map.Entry> ceilingEntry = hashRing.ceilingEntry(hash); + Map.Entry ceilingEntry = hashRing.ceilingEntry(hash); if (ceilingEntry != null) { - consumerList = ceilingEntry.getValue(); + return ceilingEntry.getValue().consumer; } else { - consumerList = hashRing.firstEntry().getValue(); + // Handle wrap-around in the hash ring, return the first consumer + return hashRing.firstEntry().getValue().consumer; } - - return consumerList.get(hash % consumerList.size()); } finally { rwLock.readLock().unlock(); } @@ -132,16 +135,27 @@ public Consumer select(int hash) { @Override public Map> getConsumerKeyHashRanges() { - Map> result = new LinkedHashMap<>(); + Map> result = new IdentityHashMap<>(); rwLock.readLock().lock(); try { + if (hashRing.isEmpty()) { + return result; + } int start = 0; - for (Map.Entry> entry: hashRing.entrySet()) { - for (Consumer consumer: entry.getValue()) { - result.computeIfAbsent(consumer, key -> new ArrayList<>()) - .add(Range.of(start, entry.getKey())); - } - start = entry.getKey() + 1; + int lastKey = 0; + for (Map.Entry entry: hashRing.entrySet()) { + Consumer consumer = entry.getValue().consumer; + result.computeIfAbsent(consumer, key -> new ArrayList<>()) + .add(Range.of(start, entry.getKey())); + lastKey = entry.getKey(); + start = lastKey + 1; + } + // Handle wrap-around in the hash ring, the first consumer will also contain the range from the last key + // to the maximum value of the hash range + Consumer firstConsumer = hashRing.firstEntry().getValue().consumer; + List ranges = result.get(firstConsumer); + if (lastKey != Integer.MAX_VALUE - 1) { + ranges.add(Range.of(lastKey + 1, Integer.MAX_VALUE - 1)); } } finally { rwLock.readLock().unlock(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapper.java new file mode 100644 index 0000000000000..2aae1d9b0622e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapper.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +/** + * A wrapper class for a Consumer instance that provides custom implementations + * of equals and hashCode methods. The equals method returns true if and only if + * the compared instance is the same instance. + * + *

The reason for this class is the custom implementation of {@link Consumer#equals(Object)}. + * Using this wrapper class will be useful in use cases where it's necessary to match a key + * in a map by instance or a value in a set by instance.

+ */ +class ConsumerIdentityWrapper { + final Consumer consumer; + + public ConsumerIdentityWrapper(Consumer consumer) { + this.consumer = consumer; + } + + /** + * Compares this wrapper to the specified object. The result is true if and only if + * the argument is not null and is a ConsumerIdentityWrapper object that wraps + * the same Consumer instance. + * + * @param obj the object to compare this ConsumerIdentityWrapper against + * @return true if the given object represents a ConsumerIdentityWrapper + * equivalent to this wrapper, false otherwise + */ + @Override + public boolean equals(Object obj) { + if (obj instanceof ConsumerIdentityWrapper) { + ConsumerIdentityWrapper other = (ConsumerIdentityWrapper) obj; + return consumer == other.consumer; + } + return false; + } + + /** + * Returns a hash code for this wrapper. The hash code is computed based on + * the wrapped Consumer instance. + * + * @return a hash code value for this object + */ + @Override + public int hashCode() { + return consumer.hashCode(); + } + + @Override + public String toString() { + return consumer.toString(); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerNameIndexTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerNameIndexTracker.java new file mode 100644 index 0000000000000..1f93313ab1b71 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerNameIndexTracker.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import java.util.HashMap; +import java.util.Map; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.commons.lang3.mutable.MutableInt; +import org.roaringbitmap.RoaringBitmap; + +/** + * Tracks the used consumer name indexes for each consumer name. + * This is used by {@link ConsistentHashingStickyKeyConsumerSelector} to get a unique "consumer name index" + * for each consumer name. It is useful when there are multiple consumers with the same name, but they are + * different consumers. The purpose of the index is to prevent collisions in the hash ring. + * + * The consumer name index serves as an additional key for the hash ring assignment. The logic keeps track of + * used "index slots" for each consumer name and assigns the first unused index when a new consumer is added. + * This approach minimizes hash collisions due to using the same consumer name. + * + * An added benefit of this tracking approach is that a consumer that leaves and then rejoins immediately will get the + * same index and therefore the same assignments in the hash ring. This improves stability since the hash assignment + * changes are minimized over time, although a better solution would be to avoid reusing the same consumer name + * in the first place. + * + * When a consumer is removed, the index is deallocated. RoaringBitmap is used to keep track of the used indexes. + * The data structure to track a consumer name is removed when the reference count of the consumer name is zero. + * + * This class is not thread-safe and should be used in a synchronized context in the caller. + */ +@NotThreadSafe +class ConsumerNameIndexTracker { + // tracks the used index slots for each consumer name + private final Map consumerNameIndexSlotsMap = new HashMap<>(); + // tracks the active consumer entries + private final Map consumerEntries = new HashMap<>(); + + // Represents a consumer entry in the tracker, including the consumer name, index, and reference count. + record ConsumerEntry(String consumerName, int nameIndex, MutableInt refCount) { + } + + /* + * Tracks the used indexes for a consumer name using a RoaringBitmap. + * A specific index slot is used when the bit is set. + * When all bits are cleared, the customer name can be removed from tracking. + */ + static class ConsumerNameIndexSlots { + private RoaringBitmap indexSlots = new RoaringBitmap(); + + public int allocateIndexSlot() { + // find the first index that is not set, if there is no such index, add a new one + int index = (int) indexSlots.nextAbsentValue(0); + if (index == -1) { + index = indexSlots.getCardinality(); + } + indexSlots.add(index); + return index; + } + + public boolean deallocateIndexSlot(int index) { + indexSlots.remove(index); + return indexSlots.isEmpty(); + } + } + + /* + * Adds a reference to the consumer and returns the index assigned to this consumer. + */ + public int increaseConsumerRefCountAndReturnIndex(ConsumerIdentityWrapper wrapper) { + ConsumerEntry entry = consumerEntries.computeIfAbsent(wrapper, k -> { + String consumerName = wrapper.consumer.consumerName(); + return new ConsumerEntry(consumerName, allocateConsumerNameIndex(consumerName), new MutableInt(0)); + }); + entry.refCount.increment(); + return entry.nameIndex; + } + + private int allocateConsumerNameIndex(String consumerName) { + return getConsumerNameIndexBitmap(consumerName).allocateIndexSlot(); + } + + private ConsumerNameIndexSlots getConsumerNameIndexBitmap(String consumerName) { + return consumerNameIndexSlotsMap.computeIfAbsent(consumerName, k -> new ConsumerNameIndexSlots()); + } + + /* + * Decreases the reference count of the consumer and removes the consumer name from tracking if the ref count is + * zero. + */ + public void decreaseConsumerRefCount(ConsumerIdentityWrapper removed) { + ConsumerEntry consumerEntry = consumerEntries.get(removed); + int refCount = consumerEntry.refCount.decrementAndGet(); + if (refCount == 0) { + deallocateConsumerNameIndex(consumerEntry.consumerName, consumerEntry.nameIndex); + consumerEntries.remove(removed, consumerEntry); + } + } + + private void deallocateConsumerNameIndex(String consumerName, int index) { + if (getConsumerNameIndexBitmap(consumerName).deallocateIndexSlot(index)) { + consumerNameIndexSlotsMap.remove(consumerName); + } + } + + /* + * Returns the currently tracked index for the consumer. + */ + public int getTrackedIndex(ConsumerIdentityWrapper wrapper) { + ConsumerEntry consumerEntry = consumerEntries.get(wrapper); + return consumerEntry != null ? consumerEntry.nameIndex : -1; + } + + int getTrackedConsumerNamesCount() { + return consumerNameIndexSlotsMap.size(); + } + + int getTrackedConsumersCount() { + return consumerEntries.size(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java index 48311c57338b5..04aafc49b47e6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java @@ -18,19 +18,27 @@ */ package org.apache.pulsar.broker.service; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; - +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; import org.apache.pulsar.client.api.Range; +import org.assertj.core.data.Offset; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -40,7 +48,7 @@ public class ConsistentHashingStickyKeyConsumerSelectorTest { @Test public void testConsumerSelect() throws ConsumerAssignException { - ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(200); String key1 = "anyKey"; Assert.assertNull(selector.select(key1.getBytes())); @@ -146,31 +154,115 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(3); List consumerName = Arrays.asList("consumer1", "consumer2", "consumer3"); List consumers = new ArrayList<>(); + long id=0; for (String s : consumerName) { - Consumer consumer = mock(Consumer.class); - when(consumer.consumerName()).thenReturn(s); + Consumer consumer = createMockConsumer(s, s, id++); selector.addConsumer(consumer); consumers.add(consumer); } + + // check that results are the same when called multiple times + assertThat(selector.getConsumerKeyHashRanges()) + .containsExactlyEntriesOf(selector.getConsumerKeyHashRanges()); + Map> expectedResult = new HashMap<>(); + assertThat(consumers.get(0).consumerName()).isEqualTo("consumer1"); expectedResult.put(consumers.get(0), Arrays.asList( - Range.of(119056335, 242013991), - Range.of(722195657, 1656011842), - Range.of(1707482098, 1914695766))); + Range.of(95615213, 440020355), + Range.of(440020356, 455987436), + Range.of(1189794593, 1264144431))); + assertThat(consumers.get(1).consumerName()).isEqualTo("consumer2"); expectedResult.put(consumers.get(1), Arrays.asList( - Range.of(0, 90164503), - Range.of(90164504, 119056334), - Range.of(382436668, 722195656))); + Range.of(939655188, 1189794592), + Range.of(1314727625, 1977451233), + Range.of(1977451234, 2016237253))); + assertThat(consumers.get(2).consumerName()).isEqualTo("consumer3"); expectedResult.put(consumers.get(2), Arrays.asList( - Range.of(242013992, 242377547), - Range.of(242377548, 382436667), - Range.of(1656011843, 1707482097))); - for (Map.Entry> entry : selector.getConsumerKeyHashRanges().entrySet()) { - System.out.println(entry.getValue()); - Assert.assertEquals(entry.getValue(), expectedResult.get(entry.getKey())); - expectedResult.remove(entry.getKey()); + Range.of(0, 95615212), + Range.of(455987437, 939655187), + Range.of(1264144432, 1314727624), + Range.of(2016237254, 2147483646))); + Map> consumerKeyHashRanges = selector.getConsumerKeyHashRanges(); + assertThat(consumerKeyHashRanges).containsExactlyInAnyOrderEntriesOf(expectedResult); + + // check that ranges are continuous and cover the whole range + List allRanges = + consumerKeyHashRanges.values().stream().flatMap(List::stream).sorted().collect(Collectors.toList()); + Range previousRange = null; + for (Range range : allRanges) { + if (previousRange != null) { + assertThat(range.getStart()).isEqualTo(previousRange.getEnd() + 1); + } + previousRange = range; + } + assertThat(allRanges.stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum()).isEqualTo(Integer.MAX_VALUE); + } + + @Test + public void testConsumersGetSufficientlyAccuratelyEvenlyMapped() + throws BrokerServiceException.ConsumerAssignException { + ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(200); + List consumers = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + // use the same name for all consumers, use toString to distinguish them + Consumer consumer = createMockConsumer("consumer", String.format("index %02d", i), i); + selector.addConsumer(consumer); + consumers.add(consumer); } - Assert.assertEquals(expectedResult.size(), 0); + printConsumerRangesStats(selector); + + int totalSelections = 10000; + + Map consumerSelectionCount = new HashMap<>(); + for (int i = 0; i < totalSelections; i++) { + Consumer selectedConsumer = selector.select(("key " + i).getBytes(StandardCharsets.UTF_8)); + consumerSelectionCount.computeIfAbsent(selectedConsumer, c -> new MutableInt()).increment(); + } + + printSelectionCountStats(consumerSelectionCount); + + int averageCount = totalSelections / consumers.size(); + int allowedVariance = (int) (0.2d * averageCount); + System.out.println("averageCount: " + averageCount + " allowedVariance: " + allowedVariance); + + for (Map.Entry entry : consumerSelectionCount.entrySet()) { + assertThat(entry.getValue().intValue()).describedAs("consumer: %s", entry.getKey()) + .isCloseTo(averageCount, Offset.offset(allowedVariance)); + } + + consumers.forEach(selector::removeConsumer); + assertThat(selector.getConsumerKeyHashRanges()).isEmpty(); + } + + private static void printSelectionCountStats(Map consumerSelectionCount) { + int totalSelections = consumerSelectionCount.values().stream().mapToInt(MutableInt::intValue).sum(); + consumerSelectionCount.entrySet().stream() + .sorted(Map.Entry.comparingByKey(Comparator.comparing(Consumer::toString))) + .forEach(entry -> System.out.println( + String.format("consumer: %s got selected %d times. ratio: %.2f%%", entry.getKey(), + entry.getValue().intValue(), + ((double) entry.getValue().intValue() / totalSelections) * 100.0d))); + } + + private static void printConsumerRangesStats(ConsistentHashingStickyKeyConsumerSelector selector) { + selector.getConsumerKeyHashRanges().entrySet().stream() + .map(entry -> Map.entry(entry.getKey(), + entry.getValue().stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum())) + .sorted(Map.Entry.comparingByKey(Comparator.comparing(Consumer::toString))) + .forEach(entry -> System.out.println( + String.format("consumer: %s total ranges size: %d ratio: %.2f%%", entry.getKey(), + entry.getValue(), + ((double) entry.getValue() / (Integer.MAX_VALUE - 1)) * 100.0d))); + } + + private static Consumer createMockConsumer(String consumerName, String toString, long id) { + // without stubOnly, the mock will record method invocations and run into OOME + Consumer consumer = mock(Consumer.class, Mockito.withSettings().stubOnly()); + when(consumer.consumerName()).thenReturn(consumerName); + when(consumer.getPriorityLevel()).thenReturn(0); + when(consumer.toString()).thenReturn(toString); + when(consumer.consumerId()).thenReturn(id); + return consumer; } // reproduces https://github.com/apache/pulsar/issues/22050 @@ -215,5 +307,243 @@ public void shouldRemoveConsumersFromConsumerKeyHashRanges() { consumers.forEach(selector::removeConsumer); // then there should be no mapping remaining Assert.assertEquals(selector.getConsumerKeyHashRanges().size(), 0); + // when consumers are removed again, should not fail + consumers.forEach(selector::removeConsumer); + } + + @Test + public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemoved() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 100; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + int hashRangeSize = Integer.MAX_VALUE; + int validationPointCount = 200; + int increment = hashRangeSize / (validationPointCount + 1); + List selectedConsumerBeforeRemoval = new ArrayList<>(); + + for (int i = 0; i < validationPointCount; i++) { + selectedConsumerBeforeRemoval.add(selector.select(i * increment)); + } + + for (int i = 0; i < validationPointCount; i++) { + Consumer selected = selector.select(i * increment); + Consumer expected = selectedConsumerBeforeRemoval.get(i); + assertThat(selected.consumerId()).as("validationPoint %d", i).isEqualTo(expected.consumerId()); + } + + Set removedConsumers = new HashSet<>(); + for (Consumer removedConsumer : consumers) { + selector.removeConsumer(removedConsumer); + removedConsumers.add(removedConsumer); + for (int i = 0; i < validationPointCount; i++) { + int hash = i * increment; + Consumer selected = selector.select(hash); + Consumer expected = selectedConsumerBeforeRemoval.get(i); + if (!removedConsumers.contains(expected)) { + assertThat(selected.consumerId()).as("validationPoint %d, removed %s, hash %d ranges %s", i, + removedConsumer.toString(), hash, selector.getConsumerKeyHashRanges()).isEqualTo(expected.consumerId()); + } + } + } + } + + @Test + public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemovedCheckHashRanges() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 25; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + Map> expected = selector.getConsumerKeyHashRanges(); + assertThat(selector.getConsumerKeyHashRanges()).as("sanity check").containsExactlyInAnyOrderEntriesOf(expected); + System.out.println(expected); + + for (Consumer removedConsumer : consumers) { + selector.removeConsumer(removedConsumer); + for (Map.Entry> entry : expected.entrySet()) { + if (entry.getKey() == removedConsumer) { + continue; + } + for (Range range : entry.getValue()) { + Consumer rangeStartConsumer = selector.select(range.getStart()); + assertThat(rangeStartConsumer).as("removed %s, range %s", removedConsumer, range) + .isEqualTo(entry.getKey()); + Consumer rangeEndConsumer = selector.select(range.getEnd()); + assertThat(rangeEndConsumer).as("removed %s, range %s", removedConsumer, range) + .isEqualTo(entry.getKey()); + assertThat(rangeStartConsumer).isSameAs(rangeEndConsumer); + } + } + expected = selector.getConsumerKeyHashRanges(); + } + } + + @Test + public void testShouldNotChangeSelectedConsumerUnnecessarilyWhenConsumerIsAddedCheckHashRanges() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 25; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + Map> expected = selector.getConsumerKeyHashRanges(); + assertThat(selector.getConsumerKeyHashRanges()).as("sanity check").containsExactlyInAnyOrderEntriesOf(expected); + + for (int i = numOfInitialConsumers; i < numOfInitialConsumers * 2; i++) { + final Consumer addedConsumer = createMockConsumer(consumerName, "index " + i, i); + selector.addConsumer(addedConsumer); + for (Map.Entry> entry : expected.entrySet()) { + if (entry.getKey() == addedConsumer) { + continue; + } + for (Range range : entry.getValue()) { + Consumer rangeStartConsumer = selector.select(range.getStart()); + if (rangeStartConsumer != addedConsumer) { + assertThat(rangeStartConsumer).as("added %s, range start %s", addedConsumer, range) + .isEqualTo(entry.getKey()); + } + Consumer rangeEndConsumer = selector.select(range.getStart()); + if (rangeEndConsumer != addedConsumer) { + assertThat(rangeEndConsumer).as("added %s, range end %s", addedConsumer, range) + .isEqualTo(entry.getKey()); + } + } + } + expected = selector.getConsumerKeyHashRanges(); + } + } + + @Test + public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 50; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + int hashRangeSize = Integer.MAX_VALUE; + int validationPointCount = 200; + int increment = hashRangeSize / (validationPointCount + 1); + List selectedConsumerBeforeRemoval = new ArrayList<>(); + + for (int i = 0; i < validationPointCount; i++) { + selectedConsumerBeforeRemoval.add(selector.select(i * increment)); + } + + for (int i = 0; i < validationPointCount; i++) { + Consumer selected = selector.select(i * increment); + Consumer expected = selectedConsumerBeforeRemoval.get(i); + assertThat(selected.consumerId()).as("validationPoint %d", i).isEqualTo(expected.consumerId()); + } + + Set addedConsumers = new HashSet<>(); + for (int i = numOfInitialConsumers; i < numOfInitialConsumers * 2; i++) { + final Consumer addedConsumer = createMockConsumer(consumerName, "index " + i, i); + selector.addConsumer(addedConsumer); + addedConsumers.add(addedConsumer); + for (int j = 0; j < validationPointCount; j++) { + int hash = j * increment; + Consumer selected = selector.select(hash); + Consumer expected = selectedConsumerBeforeRemoval.get(j); + if (!addedConsumers.contains(addedConsumer)) { + assertThat(selected.consumerId()).as("validationPoint %d, hash %d", j, hash).isEqualTo(expected.consumerId()); + } + } + } + } + + @Test + public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 25; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + Map> expected = selector.getConsumerKeyHashRanges(); + assertThat(selector.getConsumerKeyHashRanges()).as("sanity check").containsExactlyInAnyOrderEntriesOf(expected); + + selector.removeConsumer(consumers.get(0)); + selector.removeConsumer(consumers.get(numOfInitialConsumers / 2)); + selector.addConsumer(consumers.get(0)); + selector.addConsumer(consumers.get(numOfInitialConsumers / 2)); + + assertThat(selector.getConsumerKeyHashRanges()).as("ranges shouldn't change").containsExactlyInAnyOrderEntriesOf(expected); + } + + @Test + public void testConsumersReconnect() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 50; + final int validationPointCount = 200; + final List pointsToTest = pointsToTest(validationPointCount); + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + // Mark original results. + List selectedConsumersBeforeRemove = new ArrayList<>(); + for (int i = 0; i < validationPointCount; i++) { + int point = pointsToTest.get(i); + selectedConsumersBeforeRemove.add(selector.select(point)); + } + + // All consumers leave (in any order) + List randomOrderConsumers = new ArrayList<>(consumers); + Collections.shuffle(randomOrderConsumers); + for (Consumer c : randomOrderConsumers) { + selector.removeConsumer(c); + } + + // All consumers reconnect in the same order as originally + for (Consumer c : consumers) { + selector.addConsumer(c); + } + + // Check that the same consumers are selected as before + for (int j = 0; j < validationPointCount; j++) { + int point = pointsToTest.get(j); + Consumer selected = selector.select(point); + Consumer expected = selectedConsumersBeforeRemove.get(j); + assertThat(selected.consumerId()).as("validationPoint %d, hash %d", j, point).isEqualTo(expected.consumerId()); + } + } + + private List pointsToTest(int validationPointCount) { + List res = new ArrayList<>(); + int hashRangeSize = Integer.MAX_VALUE; + final int increment = hashRangeSize / (validationPointCount + 1); + for (int i = 0; i < validationPointCount; i++) { + res.add(i * increment); + } + return res; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapperTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapperTest.java new file mode 100644 index 0000000000000..75c8e6db5d2a0 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapperTest.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class ConsumerIdentityWrapperTest { + private static Consumer mockConsumer() { + return mockConsumer("consumer"); + } + + private static Consumer mockConsumer(String consumerName) { + Consumer consumer = mock(Consumer.class); + when(consumer.consumerName()).thenReturn(consumerName); + return consumer; + } + + @Test + public void testEquals() { + Consumer consumer = mockConsumer(); + assertEquals(new ConsumerIdentityWrapper(consumer), new ConsumerIdentityWrapper(consumer)); + } + + @Test + public void testHashCode() { + Consumer consumer = mockConsumer(); + assertEquals(new ConsumerIdentityWrapper(consumer).hashCode(), + new ConsumerIdentityWrapper(consumer).hashCode()); + } + + @Test + public void testEqualsAndHashCode() { + Consumer consumer1 = mockConsumer(); + Consumer consumer2 = mockConsumer(); + ConsumerIdentityWrapper wrapper1 = new ConsumerIdentityWrapper(consumer1); + ConsumerIdentityWrapper wrapper2 = new ConsumerIdentityWrapper(consumer1); + ConsumerIdentityWrapper wrapper3 = new ConsumerIdentityWrapper(consumer2); + + // Test equality + assertEquals(wrapper1, wrapper2); + assertNotEquals(wrapper1, wrapper3); + + // Test hash code + assertEquals(wrapper1.hashCode(), wrapper2.hashCode()); + assertNotEquals(wrapper1.hashCode(), wrapper3.hashCode()); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerNameIndexTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerNameIndexTrackerTest.java new file mode 100644 index 0000000000000..0f18ecce2ffb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerNameIndexTrackerTest.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class ConsumerNameIndexTrackerTest { + private ConsumerNameIndexTracker tracker; + + @BeforeMethod + public void setUp() { + tracker = new ConsumerNameIndexTracker(); + } + + private static Consumer mockConsumer() { + return mockConsumer("consumer"); + } + + + private static Consumer mockConsumer(String consumerName) { + Consumer consumer = mock(Consumer.class); + when(consumer.consumerName()).thenReturn(consumerName); + return consumer; + } + + @Test + public void testIncreaseConsumerRefCountAndReturnIndex() { + Consumer consumer1 = mockConsumer(); + Consumer consumer2 = mockConsumer(); + ConsumerIdentityWrapper wrapper1 = new ConsumerIdentityWrapper(consumer1); + ConsumerIdentityWrapper wrapper2 = new ConsumerIdentityWrapper(consumer2); + int index1 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper1); + int index2 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper2); + assertNotEquals(index1, index2); + assertEquals(index1, tracker.getTrackedIndex(wrapper1)); + assertEquals(index2, tracker.getTrackedIndex(wrapper2)); + } + + @Test + public void testTrackingReturnsStableIndexWhenRemovedAndAddedInSameOrder() { + List consumerIdentityWrappers = + IntStream.range(0, 100).mapToObj(i -> mockConsumer()).map(ConsumerIdentityWrapper::new).toList(); + Map trackedIndexes = + consumerIdentityWrappers.stream().collect(Collectors.toMap( + wrapper -> wrapper, wrapper -> tracker.increaseConsumerRefCountAndReturnIndex(wrapper))); + // stop tracking every other consumer + for (int i = 0; i < consumerIdentityWrappers.size(); i++) { + if (i % 2 == 0) { + tracker.decreaseConsumerRefCount(consumerIdentityWrappers.get(i)); + } + } + // check that others are tracked + for (int i = 0; i < consumerIdentityWrappers.size(); i++) { + ConsumerIdentityWrapper wrapper = consumerIdentityWrappers.get(i); + int trackedIndex = tracker.getTrackedIndex(wrapper); + assertEquals(trackedIndex, i % 2 == 0 ? -1 : trackedIndexes.get(wrapper)); + } + // check that new consumers are tracked with the same index + for (int i = 0; i < consumerIdentityWrappers.size(); i++) { + ConsumerIdentityWrapper wrapper = consumerIdentityWrappers.get(i); + if (i % 2 == 0) { + int trackedIndex = tracker.increaseConsumerRefCountAndReturnIndex(wrapper); + assertEquals(trackedIndex, trackedIndexes.get(wrapper)); + } + } + // check that all consumers are tracked with the original indexes + for (int i = 0; i < consumerIdentityWrappers.size(); i++) { + ConsumerIdentityWrapper wrapper = consumerIdentityWrappers.get(i); + int trackedIndex = tracker.getTrackedIndex(wrapper); + assertEquals(trackedIndex, trackedIndexes.get(wrapper)); + } + } + + @Test + public void testTrackingMultipleTimes() { + List consumerIdentityWrappers = + IntStream.range(0, 100).mapToObj(i -> mockConsumer()).map(ConsumerIdentityWrapper::new).toList(); + Map trackedIndexes = + consumerIdentityWrappers.stream().collect(Collectors.toMap( + wrapper -> wrapper, wrapper -> tracker.increaseConsumerRefCountAndReturnIndex(wrapper))); + Map trackedIndexes2 = + consumerIdentityWrappers.stream().collect(Collectors.toMap( + wrapper -> wrapper, wrapper -> tracker.increaseConsumerRefCountAndReturnIndex(wrapper))); + assertThat(tracker.getTrackedConsumerNamesCount()).isEqualTo(1); + assertThat(trackedIndexes).containsExactlyInAnyOrderEntriesOf(trackedIndexes2); + consumerIdentityWrappers.forEach(wrapper -> tracker.decreaseConsumerRefCount(wrapper)); + for (ConsumerIdentityWrapper wrapper : consumerIdentityWrappers) { + int trackedIndex = tracker.getTrackedIndex(wrapper); + assertEquals(trackedIndex, trackedIndexes.get(wrapper)); + } + consumerIdentityWrappers.forEach(wrapper -> tracker.decreaseConsumerRefCount(wrapper)); + assertThat(tracker.getTrackedConsumersCount()).isEqualTo(0); + assertThat(tracker.getTrackedConsumerNamesCount()).isEqualTo(0); + } + + @Test + public void testDecreaseConsumerRefCount() { + Consumer consumer1 = mockConsumer(); + ConsumerIdentityWrapper wrapper1 = new ConsumerIdentityWrapper(consumer1); + int index1 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper1); + assertNotEquals(index1, -1); + tracker.decreaseConsumerRefCount(wrapper1); + assertEquals(tracker.getTrackedIndex(wrapper1), -1); + } + + @Test + public void testGetTrackedIndex() { + Consumer consumer1 = mockConsumer(); + Consumer consumer2 = mockConsumer(); + ConsumerIdentityWrapper wrapper1 = new ConsumerIdentityWrapper(consumer1); + ConsumerIdentityWrapper wrapper2 = new ConsumerIdentityWrapper(consumer2); + int index1 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper1); + int index2 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper2); + assertEquals(index1, tracker.getTrackedIndex(wrapper1)); + assertEquals(index2, tracker.getTrackedIndex(wrapper2)); + } + + @Test + public void testTrackingMultipleNames() { + List consumerIdentityWrappers = + IntStream.range(0, 100).mapToObj(i -> mockConsumer("consumer" + i)).map(ConsumerIdentityWrapper::new) + .toList(); + consumerIdentityWrappers.forEach(wrapper -> tracker.increaseConsumerRefCountAndReturnIndex(wrapper)); + assertThat(tracker.getTrackedConsumerNamesCount()).isEqualTo(100); + assertThat(tracker.getTrackedConsumersCount()).isEqualTo(100); + consumerIdentityWrappers.forEach(wrapper -> tracker.decreaseConsumerRefCount(wrapper)); + assertThat(tracker.getTrackedConsumersCount()).isEqualTo(0); + assertThat(tracker.getTrackedConsumerNamesCount()).isEqualTo(0); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index dcd852f409dbb..a0054f7e71425 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -20,6 +20,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.pulsar.common.protocol.Commands.serializeMetadataAndPayload; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyInt; @@ -326,7 +327,7 @@ public void testSkipRedeliverTemporally() { redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key1"))); final List readEntries = new ArrayList<>(); readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); - readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key22"))); + readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key2"))); try { Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumers.class.getDeclaredField("totalAvailablePermits"); @@ -417,7 +418,7 @@ public void testMessageRedelivery() throws Exception { // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2 final List allEntries = new ArrayList<>(); - allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key22"))); + allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key2"))); allEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); allEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key1"))); allEntries.forEach(entry -> ((EntryImpl) entry).retain()); @@ -518,8 +519,8 @@ public void testMessageRedelivery() throws Exception { persistentDispatcher.readMoreEntries(); } - assertEquals(actualEntriesToConsumer1, expectedEntriesToConsumer1); - assertEquals(actualEntriesToConsumer2, expectedEntriesToConsumer2); + assertThat(actualEntriesToConsumer1).containsExactlyElementsOf(expectedEntriesToConsumer1); + assertThat(actualEntriesToConsumer2).containsExactlyElementsOf(expectedEntriesToConsumer2); allEntries.forEach(entry -> entry.release()); } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java index 4437ffc4ac6a2..488083f484b76 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java @@ -27,7 +27,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class Range { +public class Range implements Comparable { private final int start; private final int end; @@ -84,4 +84,13 @@ public int hashCode() { public String toString() { return "[" + start + ", " + end + "]"; } + + @Override + public int compareTo(Range o) { + int result = Integer.compare(start, o.start); + if (result == 0) { + result = Integer.compare(end, o.end); + } + return result; + } } From 53e996c43d48bc33f6b60fb007fc5b202733df1b Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 2 Oct 2024 10:12:49 -0700 Subject: [PATCH 030/327] [fix][client] Fix failover consumer-listener stuck with cumulative ack and epoch time (#23345) --- .../client/impl/MessageRedeliveryTest.java | 64 ++++++++++++++++++- .../pulsar/client/impl/ConsumerImpl.java | 3 + .../client/impl/MultiTopicsConsumerImpl.java | 7 +- 3 files changed, 70 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java index 29b06f68b64eb..e2895b1d01e9f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java @@ -18,11 +18,16 @@ */ package org.apache.pulsar.client.impl; -import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import com.google.common.collect.Sets; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -37,6 +42,8 @@ import org.apache.pulsar.client.api.BatchReceivePolicy; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageListener; import org.apache.pulsar.client.api.Messages; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; @@ -49,8 +56,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import com.google.common.collect.Sets; -import io.netty.util.concurrent.DefaultThreadFactory; @Test(groups = "broker-impl") public class MessageRedeliveryTest extends ProducerConsumerBase { @@ -539,4 +544,57 @@ public void testMultiConsumerBatchRedeliveryAddEpoch(boolean enableBatch) throws // can't receive message again assertEquals(consumer.batchReceive().size(), 0); } + + /** + * This test validates that client lib correctly increases permits of individual consumer to retrieve data in case + * of incorrect epoch for partition-topic multi-consumer. + * + * @throws Exception + */ + @Test + public void testRedeliveryWithMultiConsumerAndListenerAddEpoch() throws Exception { + final String topic = "testRedeliveryWithMultiConsumerAndListenerAddEpoch"; + final String subName = "my-sub"; + int totalMessages = 100; + admin.topics().createPartitionedTopic(topic, 2); + + Map ids = new ConcurrentHashMap<>(); + CountDownLatch latch = new CountDownLatch(totalMessages); + MessageListener msgListener = (Consumer consumer, Message msg) -> { + String id = msg.getMessageId().toString(); + consumer.acknowledgeCumulativeAsync(msg); + if (ids.put(msg.getMessageId(), id) == null) { + latch.countDown(); + } + }; + @Cleanup + Consumer newConsumer = pulsarClient.newConsumer(Schema.STRING).topic(topic).subscriptionName(subName) + .messageListener(msgListener).subscriptionType(SubscriptionType.Failover) + .receiverQueueSize(totalMessages / 10).subscribe(); + + MultiTopicsConsumerImpl consumer = (MultiTopicsConsumerImpl) newConsumer; + long epoch = consumer.getConsumerEpoch() + 1; + consumer.setConsumerEpoch(epoch); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).enableBatching(false) + .create(); + + for (int i = 0; i < totalMessages; i++) { + producer.sendAsync("test" + i); + } + producer.flush(); + + // make sure listener has not received any messages until + // we call redelivery with correct epoch + for (int i = 0; i < 2; i++) { + assertTrue(ids.isEmpty()); + Thread.sleep(1000); + } + // make epoch valid to consume redelivery message again + consumer.setConsumerEpoch(epoch - 1); + consumer.redeliverUnacknowledgedMessages(); + + latch.await(10, TimeUnit.SECONDS); + assertEquals(ids.size(), totalMessages); + } } 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 03ccbae01c276..b7010a1ddc7b4 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 @@ -1842,6 +1842,9 @@ protected void increaseAvailablePermits(ClientCnx currentCnx, int delta) { int available = AVAILABLE_PERMITS_UPDATER.addAndGet(this, delta); while (available >= getCurrentReceiverQueueSize() / 2 && !paused) { if (AVAILABLE_PERMITS_UPDATER.compareAndSet(this, available, 0)) { + if (log.isDebugEnabled()) { + log.debug("[{}] Sending permit-cmd to broker with available permits = {}", topic, available); + } sendFlowPermitsToBroker(currentCnx, available); break; } else { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 513c0101ac6ac..ff293af230838 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -270,8 +270,13 @@ private void receiveMessageFromConsumer(ConsumerImpl consumer, boolean batchR // Process the message, add to the queue and trigger listener or async callback messages.forEach(msg -> { final boolean skipDueToSeek = duringSeek; - if (isValidConsumerEpoch((MessageImpl) msg) && !skipDueToSeek) { + MessageImpl msgImpl = (MessageImpl) msg; + ClientCnx cnx = msgImpl.getCnx(); + boolean isValidEpoch = isValidConsumerEpoch(msgImpl); + if (isValidEpoch && !skipDueToSeek) { messageReceived(consumer, msg); + } else if (!isValidEpoch) { + consumer.increaseAvailablePermits(cnx); } else if (skipDueToSeek) { log.info("[{}] [{}] Skip processing message {} received during seek", topic, subscription, msg.getMessageId()); From e49d9ad01c94deea36c30ce45be0b0fe26cba16b Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 2 Oct 2024 13:01:50 -0700 Subject: [PATCH 031/327] [improve][pip] PIP-360 Add admin API to display Schema metadata (#22913) --- pip/pip-360.md | 83 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 83 insertions(+) create mode 100644 pip/pip-360.md diff --git a/pip/pip-360.md b/pip/pip-360.md new file mode 100644 index 0000000000000..21e8e18dc0531 --- /dev/null +++ b/pip/pip-360.md @@ -0,0 +1,83 @@ +# PIP-360: Admin API to display Schema metadata + +# Background knowledge + +Broker loads and initializes Schema of the topic during the topic loading. However, we have seen large number of instances and issues when broker fails to load the topic when topic schema is broken due to missing or corrupt schema ledger, index ledger or even schema data. Therefore, if broker is not able to load the topic for any reason then it is not possible to fetch schema metadata and identify which schema ledger is causing the issue because broker is storing schema metadata into binary format and there is no such API exists which shows schema metadata into readable format. So, it is very important to have an API to read schema metadata with complete information to help system admin to understand topic unavailability issues. It is also very useful to get schema metadata to build various schema related external tools which can be used by system administrator. We already have APIs for managed-ledger and bookkeeper-ledgers which are used by external tools and CLI to read binary data from metadata store and display in readable format. + + +# Motivation + +Schema is one of the important part of the topic because it also plays important part in topic availability and required to successfully load the topic, and if schema initialization failure is causing issue in topic loading then it is very important to get schema metadata information to understand schema related issues and perform appropriate actions to mitigate that issue to successfully load the topic and make it available for users. Therefore, similar to ledger metadata and managed-ledger metadata, Pulsar should have API to show schema metadata and related ledger info which can be used by tools or users to perform appropriate actions during topic availability issues or any other troubleshooting. + +# Goals +Add an .admin API under schema resource which returns schema metadata into readable format + + +# High Level Design + +This PIP will introduce REST api which will accept the topic name and return schema metadata along with ledger information of schema-ledgers and index entries. It will also add CLI support to print schema metadata for users to see it in human readable format. + + +### Public API + + +This PIP will add a new REST endpoint under Schema resource path. +``` +Path: schema/{tenant}/{namespace}/{topic}/metadata +Response code: +307, message = Current broker doesn't serve the namespace of this topic +401, message = Client is not authorized or Don't have admin permission +403, message = Client is not authenticated +404, message = Tenant or Namespace or Topic doesn't exist; or Schema is not found for +412, message = Failed to find the ownership for the topic +``` +This admin API will return below schema metadata response. + +``` +@Data +public class SchemaMetadata { + public Entry info; + public List index; + @Data + @AllArgsConstructor + @NoArgsConstructor + static class Entry { + private long ledgerId; + private long entryId; + private long version; + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("ledgerId", ledgerId) + .add("entryId", entryId) + .add("version", version) + .toString(); + } + } +} +``` + +### CLI + +This PIP will also add appropriate CLI command under Schema command to get schema metadata. +``` +bin/pulsar-admin schemas get-metadata +``` + +# Links + +Sample PR: https://github.com/apache/pulsar/pull/22938 + +* Mailing List discussion thread: +* Mailing List voting thread: From ab684a0fb9d433ab3214b6e8baba828895c07999 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 3 Oct 2024 18:42:35 +0300 Subject: [PATCH 032/327] [fix][sec] Upgrade protobuf-java to 3.25.5 (#23356) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- distribution/shell/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 1d78913849bda..61d4c2231adad 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -565,8 +565,8 @@ MIT License - com.auth0-jwks-rsa-0.22.0.jar Protocol Buffers License * Protocol Buffers - - com.google.protobuf-protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt - - com.google.protobuf-protobuf-java-util-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-3.25.5.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-util-3.25.5.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 9ab22ae83e42e..aa3853c6dd926 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -431,7 +431,7 @@ MIT License Protocol Buffers License * Protocol Buffers - - protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt + - protobuf-java-3.25.5.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/pom.xml b/pom.xml index 881a1541c5eaf..66009003aa110 100644 --- a/pom.xml +++ b/pom.xml @@ -170,7 +170,7 @@ flexible messaging model and an intuitive client API. 0.5.0 1.14.12 1.17 - 3.22.3 + 3.25.5 ${protobuf3.version} 1.56.1 1.41.0 From eee9283666cc9d84d0ddb998c279600898121d2b Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 3 Oct 2024 12:50:11 -0700 Subject: [PATCH 033/327] [fix][broker] timeout when broker registry hangs and monitor broker registry (ExtensibleLoadManagerImpl only) (#23382) --- .../pulsar/broker/admin/impl/BrokersBase.java | 11 ++- .../extensions/BrokerRegistry.java | 5 + .../extensions/BrokerRegistryImpl.java | 69 ++++++++++--- .../extensions/ExtensibleLoadManagerImpl.java | 18 +++- .../channel/ServiceUnitStateChannelImpl.java | 98 ++++++++++++++++--- .../extensions/BrokerRegistryTest.java | 33 +++++++ .../ExtensibleLoadManagerImplTest.java | 15 +++ .../channel/ServiceUnitStateChannelTest.java | 70 +++++++++---- .../apache/pulsar/client/admin/Brokers.java | 12 ++- .../client/admin/internal/BrokersImpl.java | 18 +++- 10 files changed, 299 insertions(+), 50 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index 4d0b598a8e4f1..e13cb1858f79d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -48,6 +48,7 @@ import javax.ws.rs.container.Suspended; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; +import org.apache.commons.lang.StringUtils; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService.State; @@ -368,20 +369,26 @@ public void isReady(@Suspended AsyncResponse asyncResponse) { @ApiOperation(value = "Run a healthCheck against the broker") @ApiResponses(value = { @ApiResponse(code = 200, message = "Everything is OK"), + @ApiResponse(code = 307, message = "Current broker is not the target broker"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Cluster doesn't exist"), @ApiResponse(code = 500, message = "Internal server error")}) public void healthCheck(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "Topic Version") - @QueryParam("topicVersion") TopicVersion topicVersion) { + @QueryParam("topicVersion") TopicVersion topicVersion, + @QueryParam("brokerId") String brokerId) { validateSuperUserAccessAsync() .thenAccept(__ -> checkDeadlockedThreads()) + .thenCompose(__ -> maybeRedirectToBroker( + StringUtils.isBlank(brokerId) ? pulsar().getBrokerId() : brokerId)) .thenCompose(__ -> internalRunHealthCheck(topicVersion)) .thenAccept(__ -> { LOG.info("[{}] Successfully run health check.", clientAppId()); asyncResponse.resume(Response.ok("ok").build()); }).exceptionally(ex -> { - LOG.error("[{}] Fail to run health check.", clientAppId(), ex); + if (!isRedirectException(ex)) { + LOG.error("[{}] Fail to run health check.", clientAppId(), ex); + } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java index 79dba9c63342e..d154edfbb320e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java @@ -48,6 +48,11 @@ public interface BrokerRegistry extends AutoCloseable { */ boolean isStarted(); + /** + * Return the broker has been registered. + */ + boolean isRegistered(); + /** * Register local broker to metadata store. */ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java index a13b332e6eb5f..5a8307df27a63 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java @@ -52,6 +52,8 @@ @Slf4j public class BrokerRegistryImpl implements BrokerRegistry { + private static final int MAX_REGISTER_RETRY_DELAY_IN_MILLIS = 1000; + private final PulsarService pulsar; private final ServiceConfiguration conf; @@ -77,10 +79,11 @@ protected enum State { @VisibleForTesting final AtomicReference state = new AtomicReference<>(State.Init); - public BrokerRegistryImpl(PulsarService pulsar) { + @VisibleForTesting + BrokerRegistryImpl(PulsarService pulsar, MetadataCache brokerLookupDataMetadataCache) { this.pulsar = pulsar; this.conf = pulsar.getConfiguration(); - this.brokerLookupDataMetadataCache = pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class); + this.brokerLookupDataMetadataCache = brokerLookupDataMetadataCache; this.scheduler = pulsar.getLoadManagerExecutor(); this.listeners = new ArrayList<>(); this.brokerIdKeyPath = keyPath(pulsar.getBrokerId()); @@ -99,6 +102,10 @@ public BrokerRegistryImpl(PulsarService pulsar) { pulsar.getConfig().lookupProperties()); } + public BrokerRegistryImpl(PulsarService pulsar) { + this(pulsar, pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class)); + } + @Override public synchronized void start() throws PulsarServerException { if (!this.state.compareAndSet(State.Init, State.Started)) { @@ -118,6 +125,12 @@ public boolean isStarted() { return state == State.Started || state == State.Registered; } + @Override + public boolean isRegistered() { + final var state = this.state.get(); + return state == State.Registered; + } + @Override public CompletableFuture registerAsync() { final var state = this.state.get(); @@ -127,12 +140,35 @@ public CompletableFuture registerAsync() { } log.info("[{}] Started registering self to {} (state: {})", getBrokerId(), brokerIdKeyPath, state); return brokerLookupDataMetadataCache.put(brokerIdKeyPath, brokerLookupData, EnumSet.of(CreateOption.Ephemeral)) - .thenAccept(__ -> { - this.state.set(State.Registered); - log.info("[{}] Finished registering self", getBrokerId()); + .orTimeout(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS) + .whenComplete((__, ex) -> { + if (ex == null) { + this.state.set(State.Registered); + log.info("[{}] Finished registering self", getBrokerId()); + } else { + log.error("[{}] Failed registering self", getBrokerId(), ex); + } }); } + private void doRegisterAsyncWithRetries(int retry, CompletableFuture future) { + pulsar.getExecutor().schedule(() -> { + registerAsync().whenComplete((__, e) -> { + if (e != null) { + doRegisterAsyncWithRetries(retry + 1, future); + } else { + future.complete(null); + } + }); + }, Math.min(MAX_REGISTER_RETRY_DELAY_IN_MILLIS, retry * retry * 50), TimeUnit.MILLISECONDS); + } + + private CompletableFuture registerAsyncWithRetries() { + var retryFuture = new CompletableFuture(); + doRegisterAsyncWithRetries(0, retryFuture); + return retryFuture; + } + @Override public synchronized void unregister() throws MetadataStoreException { if (state.compareAndSet(State.Registered, State.Unregistering)) { @@ -219,17 +255,26 @@ private void handleMetadataStoreNotification(Notification t) { // The registered node is an ephemeral node that could be deleted when the metadata store client's session // is expired. In this case, we should register again. final var brokerId = t.getPath().substring(LOADBALANCE_BROKERS_ROOT.length() + 1); + + CompletableFuture register; if (t.getType() == NotificationType.Deleted && getBrokerId().equals(brokerId)) { - registerAsync(); - } - if (listeners.isEmpty()) { - return; + this.state.set(State.Started); + register = registerAsyncWithRetries(); + } else { + register = CompletableFuture.completedFuture(null); } - this.scheduler.submit(() -> { - for (BiConsumer listener : listeners) { - listener.accept(brokerId, t.getType()); + // Make sure to run the listeners after re-registered. + register.thenAccept(__ -> { + if (listeners.isEmpty()) { + return; } + this.scheduler.submit(() -> { + for (BiConsumer listener : listeners) { + listener.accept(brokerId, t.getType()); + } + }); }); + } catch (RejectedExecutionException e) { // Executor is shutting down } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index d8a279b854576..abca2bb398232 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -35,8 +35,10 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -987,8 +989,12 @@ protected void monitor() { return; } + // Monitor broker registry + // Periodically check the broker registry in case metadata store fails. + validateBrokerRegistry(); + // Monitor role - // Periodically check the role in case ZK watcher fails. + // Periodically check the role in case metadata store fails. var isChannelOwner = serviceUnitStateChannel.isChannelOwner(); if (isChannelOwner) { // System topic config might fail due to the race condition @@ -1087,5 +1093,15 @@ private boolean isPersistentSystemTopicUsed() { .equals(pulsar.getConfiguration().getLoadManagerServiceUnitStateTableViewClassName()); } + private void validateBrokerRegistry() + throws ExecutionException, InterruptedException, TimeoutException { + var timeout = pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(); + var lookup = brokerRegistry.lookupAsync(brokerRegistry.getBrokerId()).get(timeout, TimeUnit.SECONDS); + if (lookup.isEmpty()) { + log.warn("Found this broker:{} has not registered yet. Trying to register it", + brokerRegistry.getBrokerId()); + brokerRegistry.registerAsync().get(timeout, TimeUnit.SECONDS); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index ce975495feb2a..49d038d512e59 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -86,11 +86,13 @@ import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceBundleFactory; import org.apache.pulsar.common.naming.NamespaceBundleSplitAlgorithm; import org.apache.pulsar.common.naming.NamespaceBundles; +import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.Reflections; @@ -108,6 +110,8 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private static final long MIN_CLEAN_UP_DELAY_TIME_IN_SECS = 0; // 0 secs to clean immediately private static final long MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS = 10; private static final long MAX_OWNED_BUNDLE_COUNT_DELAY_TIME_IN_MILLIS = 10 * 60 * 1000; + private static final long MAX_BROKER_HEALTH_CHECK_RETRY = 3; + private static final long MAX_BROKER_HEALTH_CHECK_DELAY_IN_MILLIS = 1000; private final PulsarService pulsar; private final ServiceConfiguration config; private final Schema schema; @@ -115,6 +119,7 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private final String brokerId; private final Map> cleanupJobs; private final StateChangeListeners stateChangeListeners; + private BrokerRegistry brokerRegistry; private LeaderElectionService leaderElectionService; @@ -350,6 +355,11 @@ protected LeaderElectionService getLeaderElectionService() { .get().getLeaderElectionService(); } + @VisibleForTesting + protected PulsarAdmin getPulsarAdmin() throws PulsarServerException { + return pulsar.getAdminClient(); + } + @Override public synchronized void close() throws PulsarServerException { channelState = Closed; @@ -448,6 +458,14 @@ private CompletableFuture> getActiveOwnerAsync( String serviceUnit, ServiceUnitState state, Optional owner) { + + // If this broker's registry does not exist(possibly suffering from connecting to the metadata store), + // we return the owner without its activeness check. + // This broker tries to serve lookups on a best efforts basis when metadata store connection is unstable. + if (!brokerRegistry.isRegistered()) { + return CompletableFuture.completedFuture(owner); + } + return dedupeGetOwnerRequest(serviceUnit) .thenCompose(newOwner -> { if (newOwner == null) { @@ -1255,19 +1273,25 @@ private MetadataState getMetadataState() { } private void handleBrokerCreationEvent(String broker) { - CompletableFuture future = cleanupJobs.remove(broker); - if (future != null) { - future.cancel(false); - totalInactiveBrokerCleanupCancelledCnt++; - log.info("Successfully cancelled the ownership cleanup for broker:{}." - + " Active cleanup job count:{}", - broker, cleanupJobs.size()); - } else { - if (debug()) { - log.info("No needs to cancel the ownership cleanup for broker:{}." - + " There was no scheduled cleanup job. Active cleanup job count:{}", - broker, cleanupJobs.size()); - } + + if (!cleanupJobs.isEmpty() && cleanupJobs.containsKey(broker)) { + healthCheckBrokerAsync(broker) + .thenAccept(__ -> { + CompletableFuture future = cleanupJobs.remove(broker); + if (future != null) { + future.cancel(false); + totalInactiveBrokerCleanupCancelledCnt++; + log.info("Successfully cancelled the ownership cleanup for broker:{}." + + " Active cleanup job count:{}", + broker, cleanupJobs.size()); + } else { + if (debug()) { + log.info("No needs to cancel the ownership cleanup for broker:{}." + + " There was no scheduled cleanup job. Active cleanup job count:{}", + broker, cleanupJobs.size()); + } + } + }); } } @@ -1431,6 +1455,37 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max System.currentTimeMillis() - started); } + private CompletableFuture healthCheckBrokerAsync(String brokerId) { + CompletableFuture future = new CompletableFuture<>(); + doHealthCheckBrokerAsyncWithRetries(brokerId, 0, future); + return future; + } + + private void doHealthCheckBrokerAsyncWithRetries(String brokerId, int retry, CompletableFuture future) { + try { + var admin = getPulsarAdmin(); + admin.brokers().healthcheckAsync(TopicVersion.V2, Optional.of(brokerId)) + .whenComplete((__, e) -> { + if (e == null) { + log.info("Completed health-check broker :{}", brokerId, e); + future.complete(null); + return; + } + if (retry == MAX_BROKER_HEALTH_CHECK_RETRY) { + log.error("Failed health-check broker :{}", brokerId, e); + future.completeExceptionally(FutureUtil.unwrapCompletionException(e)); + } else { + pulsar.getExecutor() + .schedule(() -> doHealthCheckBrokerAsyncWithRetries(brokerId, retry + 1, future), + Math.min(MAX_BROKER_HEALTH_CHECK_DELAY_IN_MILLIS, retry * retry * 50), + MILLISECONDS); + } + }); + } catch (PulsarServerException e) { + future.completeExceptionally(e); + } + } + private synchronized void doCleanup(String broker, boolean gracefully) { try { if (getChannelOwnerAsync().get(MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS) @@ -1444,6 +1499,23 @@ private synchronized void doCleanup(String broker, boolean gracefully) { return; } + // if not gracefully, verify the broker is inactive by health-check. + if (!gracefully) { + try { + healthCheckBrokerAsync(broker).get( + pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), SECONDS); + log.warn("Found that the broker to clean is healthy. Skip the broker:{}'s orphan bundle cleanup", + broker); + return; + } catch (Exception e) { + if (debug()) { + log.info("Failed to check broker:{} health", broker, e); + } + log.info("Checked the broker:{} health. Continue the orphan bundle cleanup", broker); + } + } + + long startTime = System.nanoTime(); log.info("Started ownership cleanup for the inactive broker:{}", broker); int orphanServiceUnitCleanupCnt = 0; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java index 28a2a18500f5f..941d0e4cbc3a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java @@ -19,7 +19,10 @@ package org.apache.pulsar.broker.loadbalance.extensions; import static org.apache.pulsar.broker.loadbalance.LoadManager.LOADBALANCE_BROKERS_ROOT; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -36,6 +39,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -48,6 +52,7 @@ import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; @@ -396,6 +401,34 @@ public void testKeyPath() { assertEquals(keyPath, LOADBALANCE_BROKERS_ROOT + "/brokerId"); } + @Test + public void testRegisterAsyncTimeout() throws Exception { + var pulsar1 = createPulsarService(); + pulsar1.start(); + pulsar1.getConfiguration().setMetadataStoreOperationTimeoutSeconds(1); + var metadataCache = mock(MetadataCache.class); + var brokerRegistry = new BrokerRegistryImpl(pulsar1, metadataCache); + + // happy case + doReturn(CompletableFuture.completedFuture(null)).when(metadataCache).put(any(), any(), any()); + brokerRegistry.start(); + + // unhappy case (timeout) + doAnswer(invocationOnMock -> { + return CompletableFuture.supplyAsync(() -> null, CompletableFuture.delayedExecutor(5, TimeUnit.SECONDS)); + }).when(metadataCache).put(any(), any(), any()); + try { + brokerRegistry.registerAsync().join(); + } catch (Exception e) { + assertTrue(e.getCause() instanceof TimeoutException); + } + + // happy case again + doReturn(CompletableFuture.completedFuture(null)).when(metadataCache).put(any(), any(), any()); + brokerRegistry.registerAsync().join(); + } + + private static BrokerRegistryImpl.State getState(BrokerRegistryImpl brokerRegistry) { return brokerRegistry.state.get(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 7871e612c847a..d8d3e5bb44ffb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -132,6 +132,7 @@ import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; @@ -2106,6 +2107,20 @@ public void compactionScheduleTest() { }); } + @Test(timeOut = 30 * 1000) + public void testMonitorBrokerRegistry() throws MetadataStoreException { + primaryLoadManager.getBrokerRegistry().unregister(); + assertFalse(primaryLoadManager.getBrokerRegistry().isRegistered()); + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(30, TimeUnit.SECONDS) + .ignoreExceptions() + .untilAsserted(() -> { // wait until true + primaryLoadManager.monitor(); + assertTrue(primaryLoadManager.getBrokerRegistry().isRegistered()); + }); + } + private static abstract class MockBrokerFilter implements BrokerFilter { @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index 92cdf61f44269..b6e38d4f6956c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -44,6 +44,7 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -89,6 +90,8 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.admin.Brokers; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; @@ -136,10 +139,14 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private BrokerRegistryImpl registry; + private PulsarAdmin pulsarAdmin; + private ExtensibleLoadManagerImpl loadManager; private final String serviceUnitStateTableViewClassName; + private Brokers brokers; + @DataProvider(name = "serviceUnitStateTableViewClassName") public static Object[][] serviceUnitStateTableViewClassName() { return new Object[][]{ @@ -174,7 +181,9 @@ protected void setup() throws Exception { admin.namespaces().createNamespace(namespaceName2); pulsar1 = pulsar; - registry = new BrokerRegistryImpl(pulsar); + registry = spy(new BrokerRegistryImpl(pulsar1)); + registry.start(); + pulsarAdmin = spy(pulsar.getAdminClient()); loadManagerContext = mock(LoadManagerContext.class); doReturn(mock(LoadDataStore.class)).when(loadManagerContext).brokerLoadDataStore(); doReturn(mock(LoadDataStore.class)).when(loadManagerContext).topBundleLoadDataStore(); @@ -207,6 +216,10 @@ protected void setup() throws Exception { childBundle31 = namespaceName2 + "/" + childBundle1Range; childBundle32 = namespaceName2 + "/" + childBundle2Range; + + brokers = mock(Brokers.class); + doReturn(CompletableFuture.failedFuture(new RuntimeException("failed"))).when(brokers) + .healthcheckAsync(any(), any()); } @BeforeMethod @@ -220,6 +233,7 @@ protected void initChannels() throws Exception { cleanMetadataState(channel1); cleanMetadataState(channel2); enableChannels(); + reset(pulsarAdmin); } @@ -719,17 +733,19 @@ public void handleMetadataSessionEventTest() throws IllegalAccessException { @Test(priority = 8) public void handleBrokerCreationEventTest() throws IllegalAccessException { var cleanupJobs = getCleanupJobs(channel1); - String broker = "broker-1"; + String broker = brokerId2; var future = new CompletableFuture(); cleanupJobs.put(broker, future); ((ServiceUnitStateChannelImpl) channel1).handleBrokerRegistrationEvent(broker, NotificationType.Created); - assertEquals(0, cleanupJobs.size()); - assertTrue(future.isCancelled()); + Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + assertEquals(0, cleanupJobs.size()); + assertTrue(future.isCancelled()); + }); + } @Test(priority = 9) - public void handleBrokerDeletionEventTest() - throws IllegalAccessException, ExecutionException, InterruptedException, TimeoutException { + public void handleBrokerDeletionEventTest() throws Exception { var cleanupJobs1 = getCleanupJobs(channel1); var cleanupJobs2 = getCleanupJobs(channel2); @@ -782,8 +798,12 @@ public void handleBrokerDeletionEventTest() System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); FieldUtils.writeDeclaredField(followerChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); + + doReturn(brokers).when(pulsarAdmin).brokers(); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); + + leaderChannel.handleBrokerRegistrationEvent(brokerId2, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(brokerId2, @@ -841,6 +861,7 @@ public void handleBrokerDeletionEventTest() 3, 0, 0); + reset(pulsarAdmin); // broker is back online leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Created); @@ -865,6 +886,7 @@ public void handleBrokerDeletionEventTest() // broker is offline again + doReturn(brokers).when(pulsarAdmin).brokers(); FieldUtils.writeDeclaredField(leaderChannel, "maxCleanupDelayTimeInSecs", 3, true); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); @@ -906,6 +928,7 @@ public void handleBrokerDeletionEventTest() 4, 0, 1); + reset(pulsarAdmin); // test unstable state channel1.publishUnloadEventAsync(new Unload(brokerId2, bundle1, Optional.of(broker))); @@ -1585,9 +1608,12 @@ public void testOverrideInactiveBrokerStateData() System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); FieldUtils.writeDeclaredField(followerChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); + + doReturn(brokers).when(pulsarAdmin).brokers(); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); + waitUntilNewOwner(channel2, releasingBundle, brokerId2); waitUntilNewOwner(channel2, childBundle11, brokerId2); waitUntilNewOwner(channel2, childBundle12, brokerId2); @@ -1600,7 +1626,7 @@ public void testOverrideInactiveBrokerStateData() // clean-up FieldUtils.writeDeclaredField(leaderChannel, "maxCleanupDelayTimeInSecs", 3 * 60, true); cleanTableViews(); - + reset(pulsarAdmin); } @Test(priority = 19) @@ -1736,13 +1762,10 @@ public void testActiveGetOwner() throws Exception { } // case 5: the owner lookup gets delayed - var spyRegistry = spy(new BrokerRegistryImpl(pulsar)); - FieldUtils.writeDeclaredField(channel1, - "brokerRegistry", spyRegistry, true); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1000, true); var delayedFuture = new CompletableFuture(); - doReturn(delayedFuture).when(spyRegistry).lookupAsync(eq(broker)); + doReturn(delayedFuture).when(registry).lookupAsync(eq(broker)); CompletableFuture.runAsync(() -> { try { Thread.sleep(500); @@ -1760,7 +1783,7 @@ public void testActiveGetOwner() throws Exception { // case 6: the owner is inactive doReturn(CompletableFuture.completedFuture(Optional.empty())) - .when(spyRegistry).lookupAsync(eq(broker)); + .when(registry).lookupAsync(eq(broker)); // verify getOwnerAsync times out start = System.currentTimeMillis(); @@ -1768,6 +1791,18 @@ public void testActiveGetOwner() throws Exception { assertTrue(ex.getCause() instanceof IllegalStateException); assertTrue(System.currentTimeMillis() - start >= 1000); + try { + // verify getOwnerAsync returns immediately when not registered + registry.unregister(); + start = System.currentTimeMillis(); + assertEquals(broker, channel1.getOwnerAsync(bundle).get().get()); + elapsed = System.currentTimeMillis() - start; + assertTrue(elapsed < 1000); + } finally { + registry.registerAsync().join(); + } + + // case 7: the ownership cleanup(no new owner) by the leader channel doReturn(CompletableFuture.completedFuture(Optional.empty())) .when(loadManager).selectAsync(any(), any(), any()); @@ -1781,6 +1816,7 @@ public void testActiveGetOwner() throws Exception { leaderChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); + doReturn(brokers).when(pulsarAdmin).brokers(); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); // verify the ownership cleanup, and channel's getOwnerAsync returns empty result without timeout @@ -1792,7 +1828,7 @@ public void testActiveGetOwner() throws Exception { waitUntilState(channel2, bundle, Init); assertTrue(System.currentTimeMillis() - start < 20_000); - + reset(pulsarAdmin); // case 8: simulate ownership cleanup(brokerId1 as the new owner) by the leader channel try { disableChannels(); @@ -1807,6 +1843,7 @@ public void testActiveGetOwner() throws Exception { FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); getCleanupJobs(leaderChannel).clear(); + doReturn(brokers).when(pulsarAdmin).brokers(); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); // verify the ownership cleanup, and channel's getOwnerAsync returns brokerId1 without timeout @@ -1817,10 +1854,8 @@ public void testActiveGetOwner() throws Exception { // test clean-up FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); - FieldUtils.writeDeclaredField(channel1, - "brokerRegistry", registry, true); cleanTableViews(); - + reset(pulsarAdmin); } @Test(priority = 21) @@ -2253,7 +2288,7 @@ private static void validateMonitorCounters(ServiceUnitStateChannel channel, } ServiceUnitStateChannelImpl createChannel(PulsarService pulsar) - throws IllegalAccessException { + throws IllegalAccessException, PulsarServerException { var tmpChannel = new ServiceUnitStateChannelImpl(pulsar); FieldUtils.writeDeclaredField(tmpChannel, "ownershipMonitorDelayTimeInSecs", 5, true); var channel = spy(tmpChannel); @@ -2261,6 +2296,7 @@ ServiceUnitStateChannelImpl createChannel(PulsarService pulsar) doReturn(loadManagerContext).when(channel).getContext(); doReturn(registry).when(channel).getBrokerRegistry(); doReturn(loadManager).when(channel).getLoadManager(); + doReturn(pulsarAdmin).when(channel).getPulsarAdmin(); var leaderElectionService = new LeaderElectionService( diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Brokers.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Brokers.java index dc0b7c9885a9a..eed73f38282ac 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Brokers.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Brokers.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.admin.PulsarAdminException.NotAuthorizedException; import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; @@ -320,10 +321,19 @@ Map getOwnedNamespaces(String cluster, String */ void healthcheck(TopicVersion topicVersion) throws PulsarAdminException; + /** + * Run a healthcheck on the target broker or on the broker. + * @param brokerId target broker id to check the health. If empty, it checks the health on the connected broker. + * + * @throws PulsarAdminException if the healthcheck fails. + */ + void healthcheck(TopicVersion topicVersion, Optional brokerId) throws PulsarAdminException; + /** * Run a healthcheck on the broker asynchronously. */ - CompletableFuture healthcheckAsync(TopicVersion topicVersion); + CompletableFuture healthcheckAsync(TopicVersion topicVersion, Optional brokerId); + /** * Trigger the current broker to graceful-shutdown asynchronously. diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java index b82c3fd0f414b..35b261b196eee 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import javax.ws.rs.client.Entity; import javax.ws.rs.client.InvocationCallback; @@ -168,26 +169,35 @@ public CompletableFuture backlogQuotaCheckAsync() { @Override @Deprecated public void healthcheck() throws PulsarAdminException { - healthcheck(TopicVersion.V1); + healthcheck(TopicVersion.V1, Optional.empty()); } @Override @Deprecated public CompletableFuture healthcheckAsync() { - return healthcheckAsync(TopicVersion.V1); + return healthcheckAsync(TopicVersion.V1, Optional.empty()); } + @Override public void healthcheck(TopicVersion topicVersion) throws PulsarAdminException { - sync(() -> healthcheckAsync(topicVersion)); + sync(() -> healthcheckAsync(topicVersion, Optional.empty())); } @Override - public CompletableFuture healthcheckAsync(TopicVersion topicVersion) { + public void healthcheck(TopicVersion topicVersion, Optional brokerId) throws PulsarAdminException { + sync(() -> healthcheckAsync(topicVersion, brokerId)); + } + + @Override + public CompletableFuture healthcheckAsync(TopicVersion topicVersion, Optional brokerId) { WebTarget path = adminBrokers.path("health"); if (topicVersion != null) { path = path.queryParam("topicVersion", topicVersion); } + if (brokerId.isPresent()) { + path = path.queryParam("brokerId", brokerId.get()); + } final CompletableFuture future = new CompletableFuture<>(); asyncGetRequest(path, new InvocationCallback() { From c2128dc4a1286d5cea8e6a1f9b8ccb49acb9684a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 3 Oct 2024 16:12:06 -0700 Subject: [PATCH 034/327] [fix] Bump commons-io:commons-io from 2.8.0 to 2.14.0 (#23393) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- distribution/shell/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 61d4c2231adad..8bcb7d7346b59 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -284,7 +284,7 @@ The Apache Software License, Version 2.0 - commons-cli-commons-cli-1.5.0.jar - commons-codec-commons-codec-1.15.jar - commons-configuration-commons-configuration-1.10.jar - - commons-io-commons-io-2.8.0.jar + - commons-io-commons-io-2.14.0.jar - commons-lang-commons-lang-2.6.jar - commons-logging-commons-logging-1.1.1.jar - org.apache.commons-commons-collections4-4.4.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index aa3853c6dd926..faad519df2ef5 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -340,7 +340,7 @@ The Apache Software License, Version 2.0 * Apache Commons - commons-codec-1.15.jar - commons-configuration-1.10.jar - - commons-io-2.8.0.jar + - commons-io-2.14.0.jar - commons-lang-2.6.jar - commons-logging-1.2.jar - commons-lang3-3.11.jar diff --git a/pom.xml b/pom.xml index 66009003aa110..70956b4d104eb 100644 --- a/pom.xml +++ b/pom.xml @@ -217,7 +217,7 @@ flexible messaging model and an intuitive client API. 2.12.1 3.11 1.10 - 2.8.0 + 2.14.0 1.15 2.1 2.1.9 From fad67613a4bbf5fa670bc18d7013eff3f44769a6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 4 Oct 2024 02:15:47 +0300 Subject: [PATCH 035/327] [fix][sec] Upgrade Avro to 1.11.4 to address CVE-2024-47561 (#23394) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- distribution/shell/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 8bcb7d7346b59..8c6e2cfa7159a 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -460,8 +460,8 @@ The Apache Software License, Version 2.0 * zt-zip - org.zeroturnaround-zt-zip-1.17.jar * Apache Avro - - org.apache.avro-avro-1.11.3.jar - - org.apache.avro-avro-protobuf-1.11.3.jar + - org.apache.avro-avro-1.11.4.jar + - org.apache.avro-avro-protobuf-1.11.4.jar * Apache Curator - org.apache.curator-curator-client-5.1.0.jar - org.apache.curator-curator-framework-5.1.0.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index faad519df2ef5..6e0bacb2e8845 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -414,8 +414,8 @@ The Apache Software License, Version 2.0 * Google Error Prone Annotations - error_prone_annotations-2.24.0.jar * Javassist -- javassist-3.25.0-GA.jar * Apache Avro - - avro-1.11.3.jar - - avro-protobuf-1.11.3.jar + - avro-1.11.4.jar + - avro-protobuf-1.11.4.jar * RE2j -- re2j-1.7.jar * Spotify completable-futures -- completable-futures-0.3.6.jar diff --git a/pom.xml b/pom.xml index 70956b4d104eb..c50357b840616 100644 --- a/pom.xml +++ b/pom.xml @@ -185,7 +185,7 @@ flexible messaging model and an intuitive client API. 3.4.0 5.18.0 1.12.638 - 1.11.3 + 1.11.4 2.10.10 2.6.0 5.1.0 From 38322a689b205fa8e4233146a2f9136081f92f26 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 3 Oct 2024 21:58:43 -0700 Subject: [PATCH 036/327] [improve][broker] PIP-327: Support force topic loading for unrecoverable errors (#21759) --- .../mledger/ManagedLedgerConfig.java | 12 ++++ .../mledger/impl/ManagedCursorImpl.java | 6 +- .../mledger/impl/ManagedCursorTest.java | 58 ++++++++++++++++++- .../pulsar/broker/ServiceConfiguration.java | 12 ++++ .../pulsar/broker/service/BrokerService.java | 1 + .../schema/BookkeeperSchemaStorage.java | 30 ++++++---- .../schema/BookkeeperSchemaStorageTest.java | 15 +++-- 7 files changed, 117 insertions(+), 17 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index a24251450b4f4..7b28990f35574 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -64,6 +64,7 @@ public class ManagedLedgerConfig { private long retentionTimeMs = 0; private long retentionSizeInMB = 0; private boolean autoSkipNonRecoverableData; + private boolean ledgerForceRecovery; private boolean lazyCursorRecovery = false; private long metadataOperationsTimeoutSeconds = 60; private long readEntryTimeoutSeconds = 120; @@ -465,6 +466,17 @@ public void setAutoSkipNonRecoverableData(boolean skipNonRecoverableData) { this.autoSkipNonRecoverableData = skipNonRecoverableData; } + /** + * Skip managed ledger failure to recover managed ledger forcefully. + */ + public boolean isLedgerForceRecovery() { + return ledgerForceRecovery; + } + + public void setLedgerForceRecovery(boolean ledgerForceRecovery) { + this.ledgerForceRecovery = ledgerForceRecovery; + } + /** * @return max unacked message ranges that will be persisted and recovered. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index b39fd231cdc06..f469b88cae8e6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -182,6 +182,7 @@ public class ManagedCursorImpl implements ManagedCursor { // Wether the current cursorLedger is read-only or writable private boolean isCursorLedgerReadOnly = true; + private boolean ledgerForceRecovery; // Stat of the cursor z-node // NOTE: Don't update cursorLedgerStat alone, @@ -332,6 +333,7 @@ public interface VoidCallback { markDeleteLimiter = null; } this.mbean = new ManagedCursorMXBeanImpl(this); + this.ledgerForceRecovery = getConfig().isLedgerForceRecovery(); } private void updateCursorLedgerStat(ManagedCursorInfo cursorInfo, Stat stat) { @@ -547,7 +549,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (log.isInfoEnabled()) { log.info("[{}] Opened ledger {} for cursor {}. rc={}", ledger.getName(), ledgerId, name, rc); } - if (isBkErrorNotRecoverable(rc)) { + if (isBkErrorNotRecoverable(rc) || ledgerForceRecovery) { log.error("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); // Rewind to oldest entry available @@ -575,7 +577,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (log.isDebugEnabled()) { log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed()); } - if (isBkErrorNotRecoverable(rc1)) { + if (isBkErrorNotRecoverable(rc1) || ledgerForceRecovery) { log.error("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc1)); // Rewind to oldest entry available diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 1067cda441f6a..8ae5a04a507b1 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -70,11 +70,14 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.Cleanup; +import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.client.PulsarMockBookKeeper; +import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; @@ -98,6 +101,7 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.api.proto.IntRange; import org.apache.pulsar.common.util.FutureUtil; @@ -4538,7 +4542,6 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { ledger.close(); } - @Test public void testReadEntriesWithSkipDeletedEntries() throws Exception { @Cleanup @@ -4795,5 +4798,58 @@ public void operationFailed(ManagedLedgerException exception) { assertEquals(cursor.getReadPosition(), markDeletedPosition.getNext()); } + @Test + void testForceCursorRecovery() throws Exception { + ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig(); + TestPulsarMockBookKeeper bk = new TestPulsarMockBookKeeper(executor); + factory = new ManagedLedgerFactoryImpl(metadataStore, bk); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setLedgerForceRecovery(true); + ManagedLedger ledger = factory.open("my_test_ledger", config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + ledger.addEntry("entry-1".getBytes(Encoding)); + long invalidLedger = -1L; + bk.setErrorCodeMap(invalidLedger, BKException.Code.BookieHandleNotAvailableException); + ManagedCursorInfo info = ManagedCursorInfo.newBuilder().setCursorsLedgerId(invalidLedger).build(); + CountDownLatch latch = new CountDownLatch(1); + MutableBoolean recovered = new MutableBoolean(false); + VoidCallback callback = new VoidCallback() { + @Override + public void operationComplete() { + recovered.setValue(true); + latch.countDown(); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + recovered.setValue(false); + latch.countDown(); + } + }; + c1.recoverFromLedger(info, callback); + latch.await(); + assertTrue(recovered.booleanValue()); + } + + class TestPulsarMockBookKeeper extends PulsarMockBookKeeper { + Map ledgerErrors = new HashMap<>(); + + public TestPulsarMockBookKeeper(OrderedExecutor orderedExecutor) throws Exception { + super(orderedExecutor); + } + + public void setErrorCodeMap(long ledgerId, int rc) { + ledgerErrors.put(ledgerId, rc); + } + + public void asyncOpenLedger(final long lId, final DigestType digestType, final byte[] passwd, + final OpenCallback cb, final Object ctx) { + if (ledgerErrors.containsKey(lId)) { + cb.openComplete(ledgerErrors.get(lId), null, ctx); + } + super.asyncOpenLedger(lId, digestType, passwd, cb, ctx); + } + } + private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 33b4fbff5f5bb..58d6444e7196a 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2249,6 +2249,18 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + " It helps when data-ledgers gets corrupted at bookkeeper and managed-cursor is stuck at that ledger." ) private boolean autoSkipNonRecoverableData = false; + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "Skip managed ledger failure to forcefully recover managed ledger." + ) + private boolean managedLedgerForceRecovery = false; + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "Skip schema ledger failure to forcefully recover topic successfully." + ) + private boolean schemaLedgerForceRecovery = false; @FieldContext( category = CATEGORY_STORAGE_ML, doc = "operation timeout while updating managed-ledger metadata." 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 bfa99eedcadce..dd722dffcfbfc 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 @@ -1970,6 +1970,7 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T .setRetentionTime(retentionPolicies.getRetentionTimeInMinutes(), TimeUnit.MINUTES); managedLedgerConfig.setRetentionSizeInMB(retentionPolicies.getRetentionSizeInMB()); managedLedgerConfig.setAutoSkipNonRecoverableData(serviceConfig.isAutoSkipNonRecoverableData()); + managedLedgerConfig.setLedgerForceRecovery(serviceConfig.isManagedLedgerForceRecovery()); managedLedgerConfig.setLazyCursorRecovery(serviceConfig.isLazyCursorRecovery()); managedLedgerConfig.setInactiveLedgerRollOverTime( serviceConfig.getManagedLedgerInactiveLedgerRolloverTimeSeconds(), TimeUnit.SECONDS); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java index 99f0249b304b3..85c8aa064581f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java @@ -528,7 +528,7 @@ private CompletableFuture readSchemaEntry( return openLedger(position.getLedgerId()) .thenCompose((ledger) -> - Functions.getLedgerEntry(ledger, position.getEntryId()) + Functions.getLedgerEntry(ledger, position.getEntryId(), config.isSchemaLedgerForceRecovery()) .thenCompose(entry -> closeLedger(ledger) .thenApply(ignore -> entry) ) @@ -560,7 +560,8 @@ private CompletableFuture addEntry(LedgerHandle ledgerHandle, SchemaStorag ledgerHandle.asyncAddEntry(entry.toByteArray(), (rc, handle, entryId, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to add entry", rc, ledgerHandle.getId(), -1)); + future.completeExceptionally(bkException("Failed to add entry", rc, ledgerHandle.getId(), -1, + config.isSchemaLedgerForceRecovery())); } else { future.complete(entryId); } @@ -582,7 +583,8 @@ private CompletableFuture createLedger(String schemaId) { LedgerPassword, (rc, handle, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to create ledger", rc, -1, -1)); + future.completeExceptionally(bkException("Failed to create ledger", rc, -1, -1, + config.isSchemaLedgerForceRecovery())); } else { future.complete(handle); } @@ -603,7 +605,8 @@ private CompletableFuture openLedger(Long ledgerId) { LedgerPassword, (rc, handle, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to open ledger", rc, ledgerId, -1)); + future.completeExceptionally(bkException("Failed to open ledger", rc, ledgerId, -1, + config.isSchemaLedgerForceRecovery())); } else { future.complete(handle); } @@ -617,7 +620,8 @@ private CompletableFuture closeLedger(LedgerHandle ledgerHandle) { CompletableFuture future = new CompletableFuture<>(); ledgerHandle.asyncClose((rc, handle, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to close ledger", rc, ledgerHandle.getId(), -1)); + future.completeExceptionally(bkException("Failed to close ledger", rc, ledgerHandle.getId(), -1, + config.isSchemaLedgerForceRecovery())); } else { future.complete(null); } @@ -648,12 +652,14 @@ public CompletableFuture> getStoreLedgerIdsBySchemaId(String schemaId } interface Functions { - static CompletableFuture getLedgerEntry(LedgerHandle ledger, long entry) { + static CompletableFuture getLedgerEntry(LedgerHandle ledger, long entry, + boolean forceRecovery) { final CompletableFuture future = new CompletableFuture<>(); ledger.asyncReadEntries(entry, entry, (rc, handle, entries, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to read entry", rc, ledger.getId(), entry)); + future.completeExceptionally(bkException("Failed to read entry", rc, ledger.getId(), entry, + forceRecovery)); } else { future.complete(entries.nextElement()); } @@ -700,7 +706,8 @@ static class LocatorEntry { } } - public static Exception bkException(String operation, int rc, long ledgerId, long entryId) { + public static Exception bkException(String operation, int rc, long ledgerId, long entryId, + boolean forceRecovery) { String message = org.apache.bookkeeper.client.api.BKException.getMessage(rc) + " - ledger=" + ledgerId + " - operation=" + operation; @@ -709,7 +716,10 @@ public static Exception bkException(String operation, int rc, long ledgerId, lon } boolean recoverable = rc != BKException.Code.NoSuchLedgerExistsException && rc != BKException.Code.NoSuchEntryException - && rc != BKException.Code.NoSuchLedgerExistsOnMetadataServerException; + && rc != BKException.Code.NoSuchLedgerExistsOnMetadataServerException + // if force-recovery is enabled then made it non-recoverable exception + // and force schema to skip this exception and recover immediately + && !forceRecovery; return new SchemaException(recoverable, message); } @@ -732,4 +742,4 @@ public static CompletableFuture ignoreUnrecoverableBKException(Completabl throw t instanceof CompletionException ? (CompletionException) t : new CompletionException(t); }); } -} +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageTest.java index d0c2e149bf438..3653c01daec37 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageTest.java @@ -21,6 +21,7 @@ import java.nio.ByteBuffer; import org.apache.bookkeeper.client.api.BKException; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; import org.apache.pulsar.common.schema.LongSchemaVersion; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.testng.annotations.Test; @@ -29,23 +30,29 @@ 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.assertTrue; @Test(groups = "broker") public class BookkeeperSchemaStorageTest { @Test public void testBkException() { - Exception ex = bkException("test", BKException.Code.ReadException, 1, -1); + Exception ex = bkException("test", BKException.Code.ReadException, 1, -1, false); assertEquals("Error while reading ledger - ledger=1 - operation=test", ex.getMessage()); - ex = bkException("test", BKException.Code.ReadException, 1, 0); + ex = bkException("test", BKException.Code.ReadException, 1, 0, false); assertEquals("Error while reading ledger - ledger=1 - operation=test - entry=0", ex.getMessage()); - ex = bkException("test", BKException.Code.QuorumException, 1, -1); + ex = bkException("test", BKException.Code.QuorumException, 1, -1, false); assertEquals("Invalid quorum size on ensemble size - ledger=1 - operation=test", ex.getMessage()); - ex = bkException("test", BKException.Code.QuorumException, 1, 0); + ex = bkException("test", BKException.Code.QuorumException, 1, 0, false); assertEquals("Invalid quorum size on ensemble size - ledger=1 - operation=test - entry=0", ex.getMessage()); + SchemaException sc = (SchemaException) bkException("test", BKException.Code.BookieHandleNotAvailableException, 1, 0, false); + assertTrue(sc.isRecoverable()); + sc = (SchemaException) bkException("test", BKException.Code.BookieHandleNotAvailableException, 1, 0, true); + assertFalse(sc.isRecoverable()); } @Test From 1e936778691b0cee54b6fe34b53ebc1593f5ae92 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 3 Oct 2024 23:11:36 -0700 Subject: [PATCH 037/327] [improve][pip] PIP-360 Add admin API to display Schema metadata (#22938) --- .../admin/impl/SchemasResourceBase.java | 9 ++++ .../broker/admin/v1/SchemasResource.java | 32 +++++++++++ .../broker/admin/v2/SchemasResource.java | 30 +++++++++++ .../schema/BookkeeperSchemaStorage.java | 20 +++++++ .../org/apache/pulsar/schema/SchemaTest.java | 26 +++++++++ .../apache/pulsar/client/admin/Schemas.java | 16 ++++++ .../common/policies/data/SchemaMetadata.java | 48 +++++++++++++++++ .../client/admin/internal/SchemasImpl.java | 18 +++++++ .../apache/pulsar/admin/cli/CmdSchemas.java | 13 +++++ .../pulsar/admin/cli/TestCmdSchema.java | 54 +++++++++++++++++++ 10 files changed, 266 insertions(+) create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SchemaMetadata.java create mode 100644 pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSchema.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java index 286366c8b5834..886db9c7abb37 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java @@ -31,6 +31,7 @@ import javax.ws.rs.core.Response; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.admin.AdminResource; +import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.web.RestException; @@ -38,6 +39,7 @@ import org.apache.pulsar.client.internal.DefaultImplementation; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.protocol.schema.GetAllVersionsSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetSchemaResponse; @@ -105,6 +107,13 @@ public CompletableFuture> getAllSchemasAsync(boolean aut }); } + public CompletableFuture getSchemaMetadataAsync(boolean authoritative) { + String schemaId = getSchemaId(); + BookkeeperSchemaStorage storage = (BookkeeperSchemaStorage) pulsar().getSchemaStorage(); + return validateOwnershipAndOperationAsync(authoritative, TopicOperation.GET_METADATA) + .thenCompose(__ -> storage.getSchemaMetadata(schemaId)); + } + public CompletableFuture deleteSchemaAsync(boolean authoritative, boolean force) { return validateDestinationAndAdminOperationAsync(authoritative) .thenCompose(__ -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/SchemasResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/SchemasResource.java index edc600707a120..0d6c3814bf863 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/SchemasResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/SchemasResource.java @@ -43,6 +43,7 @@ import org.apache.pulsar.broker.admin.impl.SchemasResourceBase; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; import org.apache.pulsar.broker.service.schema.exceptions.InvalidSchemaDataException; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.protocol.schema.DeleteSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetAllVersionsSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetSchemaResponse; @@ -170,6 +171,37 @@ public void getAllSchemas( }); } + @GET + @Path("/{tenant}/{cluster}/{namespace}/{topic}/metadata") + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Get the schema metadata of a topic", response = SchemaMetadata.class) + @ApiResponses(value = { + @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 401, message = "Client is not authorized or Don't have admin permission"), + @ApiResponse(code = 403, message = "Client is not authenticated"), + @ApiResponse(code = 404, + message = "Tenant or Namespace or Topic doesn't exist; or Schema is not found for this topic"), + @ApiResponse(code = 412, message = "Failed to find the ownership for the topic"), + @ApiResponse(code = 500, message = "Internal Server Error"), + }) + public void getSchemaMetadata( + @PathParam("tenant") String tenant, + @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, + @PathParam("topic") String topic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @Suspended final AsyncResponse response + ) { + validateTopicName(tenant, cluster, namespace, topic); + getSchemaMetadataAsync(authoritative) + .thenAccept(response::resume) + .exceptionally(ex -> { + log.error("[{}] Failed to get schema metadata for topic {}", clientAppId(), topicName, ex); + resumeAsyncResponseExceptionally(response, ex); + return null; + }); + } + @DELETE @Path("/{tenant}/{cluster}/{namespace}/{topic}/schema") @Produces(MediaType.APPLICATION_JSON) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/SchemasResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/SchemasResource.java index dd8ed58c853fa..07758436f6ca7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/SchemasResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/SchemasResource.java @@ -166,6 +166,36 @@ public void getAllSchemas( }); } + @GET + @Path("/{tenant}/{namespace}/{topic}/metadata") + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Get the schema metadata of a topic", response = GetAllVersionsSchemaResponse.class) + @ApiResponses(value = { + @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 401, message = "Client is not authorized or Don't have admin permission"), + @ApiResponse(code = 403, message = "Client is not authenticated"), + @ApiResponse(code = 404, + message = "Tenant or Namespace or Topic doesn't exist; or Schema is not found for this topic"), + @ApiResponse(code = 412, message = "Failed to find the ownership for the topic"), + @ApiResponse(code = 500, message = "Internal Server Error"), + }) + public void getSchemaMetadata( + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace, + @PathParam("topic") String topic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @Suspended final AsyncResponse response + ) { + validateTopicName(tenant, namespace, topic); + getSchemaMetadataAsync(authoritative) + .thenAccept(response::resume) + .exceptionally(ex -> { + log.error("[{}] Failed to get schema metadata for topic {}", clientAppId(), topicName, ex); + resumeAsyncResponseExceptionally(response, ex); + return null; + }); + } + @DELETE @Path("/{tenant}/{namespace}/{topic}/schema") @Produces(MediaType.APPLICATION_JSON) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java index 85c8aa064581f..f68cdd6473e48 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java @@ -52,8 +52,11 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.schema.SchemaStorageFormat.IndexEntry; +import org.apache.pulsar.broker.service.schema.SchemaStorageFormat.SchemaLocator; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.protocol.schema.SchemaStorage; import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.protocol.schema.StoredSchema; @@ -554,6 +557,23 @@ private CompletableFuture> getSchemaLocator(String schema o.map(r -> new LocatorEntry(r.getValue(), r.getStat().getVersion()))); } + public CompletableFuture getSchemaMetadata(String schema) { + return getLocator(schema).thenApply(locator -> { + if (!locator.isPresent()) { + return null; + } + SchemaLocator sl = locator.get().locator; + SchemaMetadata metadata = new SchemaMetadata(); + IndexEntry info = sl.getInfo(); + metadata.info = new SchemaMetadata.Entry(info.getPosition().getLedgerId(), info.getPosition().getEntryId(), + info.getVersion()); + metadata.index = sl.getIndexList() == null ? null + : sl.getIndexList().stream().map(i -> new SchemaMetadata.Entry(i.getPosition().getLedgerId(), + i.getPosition().getEntryId(), i.getVersion())).collect(Collectors.toList()); + return metadata; + }); + } + @NotNull private CompletableFuture addEntry(LedgerHandle ledgerHandle, SchemaStorageFormat.SchemaEntry entry) { final CompletableFuture future = new CompletableFuture<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java index ae9ea6d5ae6f4..ab82f981b5df3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java @@ -23,6 +23,7 @@ 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.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertThrows; @@ -84,6 +85,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.schema.KeyValueEncodingType; @@ -1492,4 +1494,28 @@ public SchemaStorageFormat.SchemaLocator deserialize(String path, byte[] content consumer.close(); producer.close(); } + + @Test + public void testTopicSchemaMetadata() throws Exception { + final String tenant = PUBLIC_TENANT; + final String namespace = "test-namespace-" + randomName(16); + final String topicOne = "metadata-topic"; + final String topicName = TopicName.get(TopicDomain.persistent.value(), tenant, namespace, topicOne).toString(); + + admin.namespaces().createNamespace(tenant + "/" + namespace, Sets.newHashSet(CLUSTER_NAME)); + + @Cleanup + Producer producer = pulsarClient + .newProducer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .topic(topicName).create(); + + SchemaMetadata metadata = admin.schemas().getSchemaMetadata(topicName); + + assertNotNull(metadata); + assertNotNull(metadata.info); + assertNotEquals(metadata.info.getLedgerId(), 0); + assertEquals(metadata.info.getEntryId(), 0); + assertEquals(metadata.index.size(), 1); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Schemas.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Schemas.java index 9a1eb67d2e53a..ca8bed253702f 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Schemas.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Schemas.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.protocol.schema.IsCompatibilityResponse; import org.apache.pulsar.common.protocol.schema.PostSchemaPayload; import org.apache.pulsar.common.schema.SchemaInfo; @@ -233,4 +234,19 @@ IsCompatibilityResponse testCompatibility(String topic, PostSchemaPayload schema * @param topic topic name, in fully qualified format */ CompletableFuture> getAllSchemasAsync(String topic); + + /** + * Get schema metadata of the topic. + * + * @param topic topic name, in fully qualified format + * @throws PulsarAdminException + */ + SchemaMetadata getSchemaMetadata(String topic) throws PulsarAdminException; + + /** + * Get schema metadata of the topic asynchronously. + * + * @param topic topic name, in fully qualified format + */ + CompletableFuture getSchemaMetadataAsync(String topic); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SchemaMetadata.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SchemaMetadata.java new file mode 100644 index 0000000000000..ff6ba6e86499e --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SchemaMetadata.java @@ -0,0 +1,48 @@ +/* + * 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.common.policies.data; + +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +/** + * Schema metadata info. + */ +@Data +public class SchemaMetadata { + + public Entry info; + public List index; + + @Data + @AllArgsConstructor + @NoArgsConstructor + public static class Entry { + private long ledgerId; + private long entryId; + private long version; + + @Override + public String toString() { + return String.format("ledgerId=[%d], entryId=[%d], version=[%d]", ledgerId, entryId, version); + } + } +} \ No newline at end of file diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java index 28b435ab5676b..7f2383e1e52ef 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.internal.DefaultImplementation; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.protocol.schema.DeleteSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetAllVersionsSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetSchemaResponse; @@ -276,6 +277,19 @@ public CompletableFuture> getAllSchemasAsync(String topic) { .collect(Collectors.toList())); } + @Override + public SchemaMetadata getSchemaMetadata(String topic) throws PulsarAdminException { + return sync(() -> getSchemaMetadataAsync(topic)); + } + + @Override + public CompletableFuture getSchemaMetadataAsync(String topic) { + TopicName tn = TopicName.get(topic); + WebTarget path = metadata(tn); + return asyncGetRequest(path, new FutureCallback(){}); + } + + private WebTarget schemaPath(TopicName topicName) { return topicPath(topicName, "schema"); } @@ -292,6 +306,10 @@ private WebTarget compatibilityPath(TopicName topicName) { return topicPath(topicName, "compatibility"); } + private WebTarget metadata(TopicName topicName) { + return topicPath(topicName, "metadata"); + } + private WebTarget topicPath(TopicName topic, String... parts) { final WebTarget base = topic.isV2() ? adminV2 : adminV1; WebTarget topicPath = base.path(topic.getRestPath(false)); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSchemas.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSchemas.java index ab8fdc1f01359..9131f11f3d33d 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSchemas.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSchemas.java @@ -44,6 +44,7 @@ public CmdSchemas(Supplier admin) { addCommand("delete", new DeleteSchema()); addCommand("upload", new UploadSchema()); addCommand("extract", new ExtractSchema()); + addCommand("metadata", new GetSchemaMetadata()); addCommand("compatibility", new TestCompatibility()); } @@ -77,6 +78,18 @@ void run() throws Exception { } } + @Command(description = "Get the schema for a topic") + private class GetSchemaMetadata extends CliCommand { + @Parameters(description = "persistent://tenant/namespace/topic", arity = "1") + private String topicName; + + @Override + void run() throws Exception { + String topic = validateTopicName(topicName); + print(getAdmin().schemas().getSchemaMetadata(topic)); + } + } + @Command(description = "Delete all versions schema of a topic") private class DeleteSchema extends CliCommand { @Parameters(description = "persistent://tenant/namespace/topic", arity = "1") diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSchema.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSchema.java new file mode 100644 index 0000000000000..b61ac3b8ef3d5 --- /dev/null +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSchema.java @@ -0,0 +1,54 @@ +/* + * 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.admin.cli; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.Schemas; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class TestCmdSchema { + + private PulsarAdmin pulsarAdmin; + + private CmdSchemas cmdSchemas; + + private Schemas schemas; + + @BeforeMethod + public void setup() throws Exception { + pulsarAdmin = mock(PulsarAdmin.class); + schemas = mock(Schemas.class); + when(pulsarAdmin.schemas()).thenReturn(schemas); + cmdSchemas = spy(new CmdSchemas(() -> pulsarAdmin)); + } + + @Test + public void testCmdClusterConfigFile() throws Exception { + String topic = "persistent://tenant/ns1/t1"; + cmdSchemas.run(new String[]{"metadata", topic}); + verify(schemas).getSchemaMetadata(eq(topic)); + } +} From 56200aabc56e75ca9ea5be1edb52d6c9d3f07fe5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 4 Oct 2024 18:14:42 +0300 Subject: [PATCH 038/327] [improve][ci] Continue Pulsar CI build even when Trivy scanner fails (#23397) --- .github/workflows/pulsar-ci.yaml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index ad017674ac6ee..091dab25ec696 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -890,8 +890,10 @@ jobs: run: src/check-binary-license.sh ./distribution/server/target/apache-pulsar-*-bin.tar.gz && src/check-binary-license.sh ./distribution/shell/target/apache-pulsar-shell-*-bin.tar.gz - name: Run Trivy container scan + id: trivy_scan uses: aquasecurity/trivy-action@master if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + continue-on-error: true with: image-ref: "apachepulsar/pulsar:latest" scanners: vuln @@ -902,7 +904,8 @@ jobs: - name: Upload Trivy scan results to GitHub Security tab uses: github/codeql-action/upload-sarif@v3 - if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + if: ${{ steps.trivy_scan.outcome == 'success' && github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + continue-on-error: true with: sarif_file: 'trivy-results.sarif' From aa125616aeb3d71ad23bd6bfeb651ae7f4fe6f55 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 4 Oct 2024 10:51:45 -0700 Subject: [PATCH 039/327] [fix][broker] Fix Broker was failing to create producer with broken schema ledger (#23395) --- .../pulsar/broker/service/AbstractTopic.java | 13 +++++-- .../service/schema/ClientGetSchemaTest.java | 35 ++++++++++++++++++- 2 files changed, 44 insertions(+), 4 deletions(-) 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 dce50a54db1f6..76dd277159cf4 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 @@ -37,6 +37,7 @@ import java.util.Set; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; @@ -64,6 +65,7 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; import org.apache.pulsar.broker.stats.prometheus.metrics.Summary; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.NamespaceName; @@ -666,9 +668,14 @@ protected String getSchemaId() { } @Override public CompletableFuture hasSchema() { - return brokerService.pulsar() - .getSchemaRegistryService() - .getSchema(getSchemaId()).thenApply(Objects::nonNull); + return brokerService.pulsar().getSchemaRegistryService().getSchema(getSchemaId()).thenApply(Objects::nonNull) + .exceptionally(e -> { + Throwable ex = e.getCause(); + if (ex instanceof SchemaException || !((SchemaException) ex).isRecoverable()) { + return false; + } + throw ex instanceof CompletionException ? (CompletionException) ex : new CompletionException(ex); + }); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java index 970e6b2712981..ec81f39fef92c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java @@ -20,8 +20,9 @@ import static org.apache.pulsar.common.naming.TopicName.PUBLIC_TENANT; import static org.apache.pulsar.schema.compatibility.SchemaCompatibilityCheckTest.randomName; -import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertNotNull; import java.util.ArrayList; import java.util.List; @@ -177,4 +178,36 @@ public void testSchemaFailure() throws Exception { producer.close(); consumer.close(); } + + @Test + public void testAddProducerOnDeletedSchemaLedgerTopic() throws Exception { + final String tenant = PUBLIC_TENANT; + final String namespace = "test-namespace-" + randomName(16); + final String topicOne = "test-deleted-schema-ledger"; + final String fqtnOne = TopicName.get(TopicDomain.persistent.value(), tenant, namespace, topicOne).toString(); + + //pulsar.getConfig().setManagedLedgerForceRecovery(true); + admin.namespaces().createNamespace(tenant + "/" + namespace, Sets.newHashSet("test")); + + // (1) create topic with schema + Producer producer = pulsarClient + .newProducer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .topic(fqtnOne).create(); + + producer.close(); + + String key = TopicName.get(fqtnOne).getSchemaName(); + BookkeeperSchemaStorage schemaStrogate = (BookkeeperSchemaStorage) pulsar.getSchemaStorage(); + long schemaLedgerId = schemaStrogate.getSchemaLedgerList(key).get(0); + + // (2) break schema locator by deleting schema-ledger + schemaStrogate.getBookKeeper().deleteLedger(schemaLedgerId); + + admin.topics().unload(fqtnOne); + + Producer producerWihtoutSchema = pulsarClient.newProducer().topic(fqtnOne).create(); + + assertNotNull(producerWihtoutSchema); + } } From 64e9687bb55c42a28f77dca73fdbd68f7f390ca4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 5 Oct 2024 00:26:30 +0300 Subject: [PATCH 040/327] [improve][pip] PIP-379: Key_Shared Draining Hashes for Improved Message Ordering (#23309) --- pip/pip-379.md | 407 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 407 insertions(+) create mode 100644 pip/pip-379.md diff --git a/pip/pip-379.md b/pip/pip-379.md new file mode 100644 index 0000000000000..3215bb541f11e --- /dev/null +++ b/pip/pip-379.md @@ -0,0 +1,407 @@ +# PIP-379: Key_Shared Draining Hashes for Improved Message Ordering + +## Background Knowledge + +Apache Pulsar's Key_Shared subscription mode is designed to provide ordered message delivery on a per-key basis while allowing multiple consumers to process messages concurrently. This mode is particularly useful in scenarios where maintaining message order for specific keys is crucial, but overall throughput can be improved by parallelizing message consumption across multiple consumers. + +Key concepts: + +- **Key_Shared subscription**: A subscription mode that maintains message ordering per key while allowing multiple consumers. +- **Hash ranges**: In AUTO_SPLIT mode, the hash space is divided among active consumers to distribute message processing. +- **Pending messages**: Messages that have been sent to a consumer but not yet acknowledged (also called "pending acks" or "unacknowledged messages"). + +### Current contract of preserving ordering + +The Key_Shared subscription is described in the [Pulsar documentation](https://pulsar.apache.org/docs/concepts-messaging/#key_shared). + +For this PIP, the most important detail is the "Preserving order of processing" section. +There are recent changes in this section that apply to the master branch of Pulsar and, therefore, to the upcoming Pulsar 4.0. The changes were made as part of ["PIP-282: Change definition of the recently joined consumers position"](https://github.com/apache/pulsar/blob/master/pip/pip-282.md). + +[PIP-282 (master branch / Pulsar 4.0) version of the "Preserving order of processing" section](https://pulsar.apache.org/docs/next/concepts-messaging/#preserving-order-of-processing): + +> Key_Shared Subscription type guarantees a key will be processed by a *single* consumer at any given time. When a new consumer is connected, some keys will change their mapping from existing consumers to the new consumer. Once the connection has been established, the broker will record the current `lastSentPosition` and associate it with the new consumer. The `lastSentPosition` is a marker indicating that messages have been dispatched to the consumers up to this point. The broker will start delivering messages to the new consumer *only* when all messages up to the `lastSentPosition` have been acknowledged. This will guarantee that a certain key is processed by a single consumer at any given time. The trade-off is that if one of the existing consumers is stuck and no time-out was defined (acknowledging for you), the new consumer won't receive any messages until the stuck consumer resumes or gets disconnected. + +[Previous version (applies to Pulsar 3.x) of the "Preserving order of processing" section](https://pulsar.apache.org/docs/3.3.x/concepts-messaging/#preserving-order-of-processing): + +> Key Shared Subscription type guarantees a key will be processed by a *single* consumer at any given time. When a new consumer is connected, some keys will change their mapping from existing consumers to the new consumer. Once the connection has been established, the broker will record the current read position and associate it with the new consumer. The read position is a marker indicating that messages have been dispatched to the consumers up to this point, and after it, no messages have been dispatched yet. The broker will start delivering messages to the new consumer *only* when all messages up to the read position have been acknowledged. This will guarantee that a certain key is processed by a single consumer at any given time. The trade-off is that if one of the existing consumers is stuck and no time-out was defined (acknowledging for you), the new consumer won't receive any messages until the stuck consumer resumes or gets disconnected. + +## Motivation + +The current implementation of Key_Shared subscriptions faces several challenges: + +1. **Complex Contract of Preserving Ordering**: The current contract of preserving ordering is hard to understand and contains a fundamental problem. It explains a solution and then ties the guarantee to the provided solution. It could be interpreted that there's a guarantee as long as this solution is able to handle the case. +2. **Incomplete Ordering Contract Fulfillment**: The current contract seems to make a conditional guarantee that a certain key is processed by a single consumer at any given time. Outside of the described solution in the contract, the current implementation struggles to consistently prevent messages from being sent to another consumer while pending on the original consumer. While Key_Shared subscriptions aim to preserve message ordering per key, the current implementation may not always achieve this, especially during consumer changes. There's a potential corner case reported in [issue #23307](https://github.com/apache/pulsar/issues/23307). +3. **Usability Issues**: Understanding the current system and detecting the reason why messages get blocked is time-consuming and difficult. +4. **Unnecessary Message Blocking**: The current implementation blocks delivery for all messages when any hash range is blocked, even if other keys could be processed independently. This leads to suboptimal utilization of consumers and increased latency for messages that could otherwise be processed. +5. **Observability Challenges**: The current implementation lacks clear visibility into the consuming state when processing gets stuck, making it harder to build automation for detecting and mitigating issues. +6. **Complexity**: The existing solution for managing "recently joined consumers" is overly complex, making the system harder to maintain and debug. + +## Goals + +### In Scope + +- Clarify and fulfill the key-ordered message delivery contract for Key_Shared AUTO_SPLIT mode. +- Fix current issues where messages are sent out-of-order or when a single key is outstanding in multiple consumers at a time. +- Improve the handling of unacknowledged messages to prevent indefinite blocking and consumers getting stuck. +- Minimize memory usage for pending message tracking, eliminating PIP-282's "sent positions" tracking. +- Implement a new "draining hashes" concept to efficiently manage message ordering in Key_Shared subscriptions. +- Enhance the reliability, usability, and scalability of Key_Shared subscriptions. +- Improve observability of Key_Shared subscriptions to aid in troubleshooting and automation. +- Ensure strict ordering guarantees for messages with the same key, even during consumer changes. + +### Out of Scope + +- Changes to other subscription types (Exclusive, Failover, Shared). +- Adding support key based ordering guarantees when negative acknowledgements are used + +## High-Level Design + +### Updated contract of preserving ordering + +The "Preserving order of processing" section of the Key_Shared documentation would be updated to contain this contract: + +_In Key_Shared subscriptions, messages with the same key are delivered and allowed to be in unacknowledged state to only one consumer at a time._ + +When new consumers join or leave, the consumer handling a message key can change when the default AUTO_SPLIT mode is used, but only after all pending messages for a particular key are acknowledged or the original consumer disconnects. + +The Key_Shared subscription doesn't prevent using any methods in the consumer API. For example, the application might call `negativeAcknowledge` or the `redeliverUnacknowledgedMessages` method. When messages are scheduled for delivery due to these methods, they will get redelivered as soon as possible. There's no ordering guarantee in these cases, however the guarantee of delivering a message key to a single consumer at a time will continue to be preserved. + +### Computer Science Perspective: Invariants + +Wikipedia tells us about [invariants](https://en.wikipedia.org/wiki/Invariant_(mathematics)#Invariants_in_computer_science): "In computer science, an invariant is a logical assertion that is always held to be true during a certain phase of execution of a computer program." + +The contract _"In Key_Shared subscriptions, messages with the same key are delivered and allowed to be in an unacknowledged state to only one consumer at a time."_ can be seen as an invariant for Key_Shared subscriptions. It is something that must always be held true for Key_Shared subscriptions. The design and implementation in PIP-379 focuses on ensuring this. + +### Future work in needed for supporting key-based ordering with negative acknowledgements + +The updated contract explicitly states that it is not possible to retain key-based ordering of messages when negative acknowledgements are used. Changing this is out of scope for PIP-379. A potential future solution for handling this would be to modify the client so that when a message is negatively acknowledged, it would also reject all further messages with the same key until the original message gets redelivered. It's already possible to attempt to implement this in client-side code. However, a proper solution would require support on the broker side to block further delivery of the specific key when there are pending negatively acknowledged messages until all negatively acknowledged messages for that particular key have been acknowledged by the consumer. This solution is out of scope for PIP-379. A future implementation to address these problems could build upon PIP-379 concepts such as "draining hashes" and extend that to cover the negative acknowledgement scenarios. + +### High-Level implementation plan + +The proposed solution introduces a "draining hashes" concept to efficiently manage message ordering in Key_Shared subscriptions: + +**1. When consumer hash ranges change (e.g., a consumer joins or leaves), affected hashes of pending messages are added to a "draining hashes" set.** + +Pending messages of the consumer are iterated, and if the hash of a pending message belongs to one of the impacted ranges, the hash gets added to the "draining hashes" tracker. + +Code example to illustrate the implementation: + +```java + private synchronized void registerDrainingHashes(Consumer skipConsumer, + Map> impactedRangesByConsumer) { + for (Map.Entry> entry : impactedRangesByConsumer.entrySet()) { + Consumer c = entry.getKey(); + if (c != skipConsumer) { + // perf optimization: convert the set to an array to avoid iterator allocation in the pending acks loop + Range[] ranges = entry.getValue().toArray(new Range[0]); + // add all pending acks in the impacted hash ranges to the draining hashes tracker + c.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + for (Range range : ranges) { + if (range.contains(stickyKeyHash)) { + // add the pending ack to the draining hashes tracker if the hash is in the range + drainingHashesTracker.addEntry(c, stickyKeyHash); + break; + } + // Since ranges are sorted, stop checking further ranges if the start of the current range is + // greater than the stickyKeyHash. + if (range.getStart() > stickyKeyHash) { + break; + } + } + }); + } + } + } +``` + +**2. Following messages with hashes in the "draining hashes" set are blocked from further delivery until pending messages are processed.** + +Code example to illustrate the implementation: + +```java + // If the hash is draining, do not send the message + if (drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash)) { + return false; + } +``` + +**3. A reference counter tracks pending messages for each hash in the "draining hashes" set.** + +Code example to illustrate the implementation: + +```java + // optimize the memory consumption of the map by using primitive int keys + private final Int2ObjectOpenHashMap drainingHashes = new Int2ObjectOpenHashMap<>(); + + public static class DrainingHashEntry { + private final Consumer consumer; + private int refCount; + private int blockedCount; + + DrainingHashEntry(Consumer consumer) { + this.consumer = consumer; + } + + public Consumer getConsumer() { + return consumer; + } + + void incrementRefCount() { + refCount++; + } + + boolean decrementRefCount() { + return --refCount == 0; + } + + void incrementBlockedCount() { + blockedCount++; + } + + boolean isBlocking() { + return blockedCount > 0; + } + } +``` + +The memory consumption estimate for tracking a hash is 52 bytes: +key: 16 bytes (object header) + 4 bytes (int) = 20 bytes +entry: 16 bytes (object header) + 8 bytes (long) + 4 bytes (int) + 4 bytes (int) = 32 bytes + +Although the estimate is 52 bytes per entry, calculations have been made with 80 bytes per entry to account for possible additional overheads such as memory alignment and the overhead of the Int2ObjectOpenHashMap. + +Memory usage estimate for each subscription after there have been consumer changes: + +- Worst case (all 64k hashes draining for a subscription): about 5MB +- Practical case (less than 1000 hashes draining): less than 80 kilobytes +- For 10,000 draining hashes: about 800 kB + +The memory usage of draining hashes tracking will go down to 0 after all hashes have "drained" and are no longer blocked. This memory usage isn't an overhead that applies at all times. + +The hash range size is reduced to 65535 (2^16-1) from the current 2^31-1 (Integer.MAX_VALUE) in ConsistentHashingStickyKeyConsumerSelector to reduce the worst-case memory consumption. Reducing the hash range size won't significantly impact the accuracy of distributing messages across connected consumers. The proof-of-concept implementation of PIP-379 includes the changes to reduce the hash range size. + +**4. As messages are acknowledged or consumers disconnect and therefore get removed from pending messages, the reference counter is decremented.** + +Individual acks are removed in Consumer's `removePendingAcks` method: + +```java + private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position) { + PendingAcksMap ownedConsumerPendingAcks = ackOwnedConsumer.getPendingAcks(); + if (!ownedConsumerPendingAcks.remove(position.getLedgerId(), position.getEntryId())) { + // Message was already removed by the other consumer + return false; + } +``` + +When the `remove` method in `PendingAcksMap` is called, it will use the `PendingAcksMap.PendingAcksRemoveHandler` callback method `handleRemoving` provided by the dispatcher to trigger the removal also from the `DrainingHashesTracker`: + +```java + consumer.setPendingAcksRemoveHandler(new PendingAcksMap.PendingAcksRemoveHandler() { + @Override + public void handleRemoving(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash, + boolean closing) { + drainingHashesTracker.reduceRefCount(consumer, stickyKeyHash, closing); + } + +``` + +Also when a consumer disconnects, hashes of pending acks are removed. This happens in the `PersistentDispatcherMultipleConsumers`'s `removeConsumer` consumer method: + +```java + consumer.getPendingAcks().forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> { + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + }); +``` + +`PendingAcksMap`'s `forEachAndClose` method will trigger removals from `DrainingHashesTracker` using the `PendingAcksMap.PendingAcksRemoveHandler` callback method `handleRemoving` after processing each entry. This is how the `DrainingHashesTracker` stays in sync with the `PendingAcksMap` state without having the need to add all logic to `PendingAcksMap`. This is about following the "separation of concerns" design principle where each class handles a specific concern. + +**5. When the reference counter reaches zero, the hash is removed from the set, allowing new message delivery. The dispatcher is notified about this so that the delivery of the blocked messages can occur. Unblocked hashes are batched together to prevent a new notification for each call. This is handled with the `keySharedUnblockingIntervalMs` configuration setting.** + +In the implementation, this is handled in the DrainingHashesTracker's reduceRefCount method: + +```java + // code example is simplified for focus on the essential details + + public synchronized void reduceRefCount(Consumer consumer, int stickyHash) { + DrainingHashEntry entry = drainingHashes.get(stickyHash); + if (entry == null) { + return; + } + if (entry.decrementRefCount()) { + DrainingHashEntry removed = drainingHashes.remove(stickyHash); + if (removed.isBlocking()) { + unblockingHandler.stickyKeyHashUnblocked(stickyHash); + } + } + } +``` + +The `isBlocking()` method of `DrainingHashEntry` returns true when delivery was attempted for that hash, indicating a need to unblock it when it's removed. +The dispatcher is notified via the `unblockingHandler.stickyKeyHashUnblocked(stickyHash)` callback. The implementation simply schedules a read, batching all calls together, and then calls `readMoreEntries` in the dispatcher. + +```java + // code example is simplified for focus on the essential details + + private void stickyKeyHashUnblocked(int stickyKeyHash) { + reScheduleReadInMs(keySharedUnblockingIntervalMsSupplier.getAsLong()); + } + + protected void reScheduleReadInMs(long readAfterMs) { + if (isRescheduleReadInProgress.compareAndSet(false, true)) { + Runnable runnable = () -> { + isRescheduleReadInProgress.set(false); + readMoreEntries(); + }; + topic.getBrokerService().executor().schedule(runnable, readAfterMs, TimeUnit.MILLISECONDS); + } + } +``` + +**6. Consumer hash assignments may change multiple times, and a draining hash might be reassigned to the original consumer.** + +The draining hash data structure contains information about the draining consumer. When a message is attempted for delivery, the system can check if the target consumer is the same as the draining consumer. If they match, there's no need to block the hash. The implementation should also remove such hashes from the draining hashes set. This "lazy" approach reduces the need for actively scanning all draining hashes whenever hash assignments change. + +This is handled in the `DrainingHashesTracker` + +```java + public synchronized boolean shouldBlockStickyKeyHash(Consumer consumer, int stickyKeyHash) { + DrainingHashEntry entry = drainingHashes.get(stickyKeyHash); + // if the entry is not found, the hash is not draining. Don't block the hash. + if (entry == null) { + return false; + } + // hash has been reassigned to the original consumer, remove the entry + // and don't block the hash + if (entry.getConsumer() == consumer) { + drainingHashes.remove(stickyKeyHash, entry); + return false; + } + // increment the blocked count which is used to determine if the hash is blocking + // dispatching to other consumers + entry.incrementBlockedCount(); + // block the hash + return true; + } +``` + +**7. When sending out messages, there are potential race conditions that could allow the delivery of a message that should be blocked.** + +This could happen when a consumer is added while reading and sending messages are already in progress. In PIP-379, the sending process has been modified to perform a check when adding the message to the pending acknowledgments map. There are also additional locks in the pending acks handling which prevent race conditions. + +`addPendingAckIfAllowed` method in `PendingAcksMap` class: + +```java + public boolean addPendingAckIfAllowed(long ledgerId, long entryId, int batchSize, int stickyKeyHash) { + try { + writeLock.lock(); + // prevent adding sticky hash to pending acks if the PendingAcksMap has already been closed + // and there's a race condition between closing the consumer and sending new messages + if (closed) { + return false; + } + // prevent adding sticky hash to pending acks if it's already in draining hashes + // to avoid any race conditions that would break consistency + PendingAcksAddHandler pendingAcksAddHandler = pendingAcksAddHandlerSupplier.get(); + if (pendingAcksAddHandler != null + && !pendingAcksAddHandler.handleAdding(consumer, ledgerId, entryId, stickyKeyHash)) { + return false; + } + Long2ObjectSortedMap ledgerPendingAcks = + pendingAcks.computeIfAbsent(ledgerId, k -> new Long2ObjectRBTreeMap<>()); + ledgerPendingAcks.put(entryId, IntIntPair.of(batchSize, stickyKeyHash)); + return true; + } finally { + writeLock.unlock(); + } + } +``` + +This `addPendingAckIfAllowed` method is called from Consumer's `sendMessages` method: + +```java + boolean sendingAllowed = + pendingAcks.addPendingAckIfAllowed(entry.getLedgerId(), entry.getEntryId(), batchSize, stickyKeyHash); + if (!sendingAllowed) { + // sending isn't allowed when pending acks doesn't accept adding the entry + // this happens when Key_Shared draining hashes contains the stickyKeyHash + // because of race conditions, it might be resolved at the time of sending + totalEntries--; + entries.set(i, null); + entry.release(); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Skipping sending of {}:{} ledger entry with batchSize of {} since adding" + + " to pending acks failed in broker.service.Consumer for consumerId: {}", + topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize, + consumerId); + } +``` + +If sending isn't allowed, the entry will be skipped from delivery. The `PendingAcksAddHandler` callback will add the message to redelivery if this is the case. +The callback maps to `handleAddingPendingAck` in the dispatcher (`PersistentStickyKeyDispatcherMultipleConsumers`). + +```java + private boolean handleAddingPendingAck(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash) { + DrainingHashesTracker.DrainingHashEntry drainingHashEntry = drainingHashesTracker.getEntry(stickyKeyHash); + if (drainingHashEntry != null && drainingHashEntry.getConsumer() != consumer) { + log.warn("[{}] Another consumer id {} is already draining hash {}. Skipping adding {}:{} to pending acks " + + "for consumer {}. Adding the message to replay.", + getName(), drainingHashEntry.getConsumer(), stickyKeyHash, ledgerId, entryId, consumer); + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + // block message from sending + return false; + } + if (recentReadTypeInSending == ReadType.Normal && redeliveryMessages.containsStickyKeyHash(stickyKeyHash)) { + log.warn("[{}] Sticky hash {} is already in the replay queue. " + + "Skipping adding {}:{} to pending acks. Adding the message to replay.", + getName(), stickyKeyHash, ledgerId, entryId); + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + // block message from sending + return false; + } + // allow adding the message to pending acks and sending the message to the consumer + return true; + } +``` + +This logic will prevent any inconsistency when consumers get added or removed and hash ranges change while the sending of messages is already in progress. It will ensure that the view on pending acknowledgments is consistent so that the tracking of draining hashes will also be consistent in all cases. In addition, this logic will block hashes of messages that have recently been added to the redelivery queue and therefore, for message ordering reasons, should get delivered before any further message delivery happens. + +**Summary** + +This high-level design approach will meet the updated contract of preserving ordering: _"In Key_Shared subscriptions, messages with the same key are delivered and allowed to be in an unacknowledged state to only one consumer at a time."_ + +It also minimizes the impact on performance and memory usage. **The tracking only comes into play during transition states.** When consumers have been connected for a longer duration and all draining hashes have been removed, there won't be a need to check any special rules or maintain any extra state. **When the draining hashes are empty, lookups will essentially be no-ops and won't consume CPU or memory resources.** + +## Public-facing Changes + +### Topic Stats Changes & Observability + +Topic stats for the removed PIP-282 "recently joined consumers"/"last sent position" solution are removed: +- `lastSentPositionWhenJoining` field for each consumer +- `consumersAfterMarkDeletePosition` field for each Key_Shared subscription +- `individuallySentPositions` field for each Key_Shared subscription + +New topic stats will be added to monitor the "draining hashes" feature at the subscription level and consumer level: +1. `draining_hashes_count`: The current number of hashes in the draining state. +2. `draining_hashes_pending_messages`: The total number of pending messages for all draining hashes. +3. `draining_hashes_cleared_total`: The total number of hashes cleared from the draining state. +4. `draining_hashes`: Details at the hash level (available at the consumer level to reduce redundancy of information) + - hash + - number of pending messages + +For improved observability, a separate REST API for listing all pending messages ("pending acks") for a consumer will be considered. This API would allow querying which messages are currently part of a draining hash, providing a way to identify specific message IDs of messages that are holding onto a specific hash and blocking delivery to another consumer. + +## Backward & Forward Compatibility + +The "draining hashes" feature doesn't introduce backward or forward compatibility issues. The state is handled at runtime, and the changes are on the broker side without changes to the client protocol. + +Slightly unrelated to PIP-379 changes, there's a need to ensure that users upgrading from Pulsar 3.x can revert to the "recently joined consumers" logic (before PIP-282) in case of possible regressions caused by PIP-379. Since PIP-282 is also new in Pulsar 4.0.0, there needs to be a feature flag that toggles between the PIP-379 implementation for Key_Shared and the "recently joined consumers" logic before PIP-282. Implemention details for this feature toggle can be handled in the pull request for implementing this. + +## Links + +- Mailing List discussion thread: https://lists.apache.org/thread/l5zjq0fb2dscys3rsn6kfl7505tbndlx +- Mailing List voting thread: https://lists.apache.org/thread/z1kgo34qfkkvdnn3l007bdvjr3qqf4rw +- PIP-379 implementation PR: https://github.com/apache/pulsar/pull/23352 + +- [PIP-282: Change definition of the recently joined consumers position](https://github.com/apache/pulsar/blob/master/pip/pip-282.md) +- [Pulsar issue #23307: Message ordering isn't retained in Key_Shared AUTO_SPLIT mode in a rolling restart type of test scenario](https://github.com/apache/pulsar/issues/23307) +- [Pulsar issue #21199: Key_Shared subscription gets stuck after consumer reconnects](https://github.com/apache/pulsar/issues/21199) \ No newline at end of file From 6bca70b1380fd97a0a1ea23bdc0bf918bd6defee Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 4 Oct 2024 16:09:48 -0700 Subject: [PATCH 041/327] [fix][client] Add more info while throwing reader creation exception (#23401) --- .../java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java index ef230475be53b..d0ab90068ed31 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java @@ -92,7 +92,8 @@ public CompletableFuture> createAsync() { return FutureUtil .failedFuture(new IllegalArgumentException( "Start message id or start message from roll back must be specified but they cannot be" - + " specified at the same time")); + + " specified at the same time. MessageId =" + conf.getStartMessageId() + + ", rollback seconds =" + conf.getStartMessageFromRollbackDurationInSec())); } if (conf.getStartMessageFromRollbackDurationInSec() > 0) { From b9ededc30e8f7d73e1226d22d327e10968df42d4 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 4 Oct 2024 16:10:00 -0700 Subject: [PATCH 042/327] [fix][broker] Fix delivery-test with unnecessary invocation-count (#23399) --- .../org/apache/pulsar/client/impl/MessageRedeliveryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java index e2895b1d01e9f..7269df3b6b8b2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java @@ -483,7 +483,7 @@ public void testMultiConsumerRedeliveryAddEpoch(boolean enableBatch) throws Exce assertNull(message); } - @Test(dataProvider = "enableBatch", invocationCount = 10) + @Test(dataProvider = "enableBatch") public void testMultiConsumerBatchRedeliveryAddEpoch(boolean enableBatch) throws Exception{ final String topic = "testMultiConsumerBatchRedeliveryAddEpoch"; From 6c7ec4c38c8dce8351096a494b412952be5cc77b Mon Sep 17 00:00:00 2001 From: vineeth1995 Date: Sun, 6 Oct 2024 22:58:25 -0700 Subject: [PATCH 043/327] [feat] [broker] Add broker health check status into prometheus metrics (#20147) --- conf/broker.conf | 2 + .../pulsar/broker/ServiceConfiguration.java | 6 +++ .../pulsar/broker/admin/impl/BrokersBase.java | 50 ++++++++++++------- .../pulsar/broker/service/BrokerService.java | 23 +++++++++ .../stats/BrokerOperabilityMetrics.java | 18 ++++++- .../service/PersistentTopicE2ETest.java | 32 ++++++++++++ .../broker/stats/PrometheusMetricsTest.java | 17 ++++++- 7 files changed, 128 insertions(+), 20 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 617e202e5ec65..e745fcb2b0a8f 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1689,6 +1689,8 @@ exposePublisherStats=true statsUpdateFrequencyInSecs=60 statsUpdateInitialDelayInSecs=60 +healthCheckMetricsUpdateTimeInSeconds=-1 + # Enable expose the precise backlog stats. # Set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. # Default is false. diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 58d6444e7196a..81073b1731b24 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -3280,6 +3280,12 @@ public double getLoadBalancerBandwidthOutResourceWeight() { doc = "Stats update initial delay in seconds" ) private int statsUpdateInitialDelayInSecs = 60; + @FieldContext( + category = CATEGORY_METRICS, + minValue = -1, + doc = "HealthCheck update frequency in seconds. Disable health check with value -1 (Default value -1)" + ) + private int healthCheckMetricsUpdateTimeInSeconds = -1; @FieldContext( category = CATEGORY_METRICS, doc = "If true, aggregate publisher stats of PartitionedTopicStats by producerName" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index e13cb1858f79d..da4cee7b4651c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -51,6 +51,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.PulsarService.State; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.admin.AdminResource; @@ -422,26 +423,35 @@ public static String getHeartbeatTopicName(String brokerId, ServiceConfiguration } private CompletableFuture internalRunHealthCheck(TopicVersion topicVersion) { - String brokerId = pulsar().getBrokerId(); + return internalRunHealthCheck(topicVersion, pulsar(), clientAppId()); + } + + + public static CompletableFuture internalRunHealthCheck(TopicVersion topicVersion, PulsarService pulsar, + String clientAppId) { + NamespaceName namespaceName = (topicVersion == TopicVersion.V2) + ? NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(), pulsar.getConfiguration()) + : NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), pulsar.getConfiguration()); + String brokerId = pulsar.getBrokerId(); final String topicName = - getHeartbeatTopicName(brokerId, pulsar().getConfiguration(), (topicVersion == TopicVersion.V2)); - LOG.info("[{}] Running healthCheck with topic={}", clientAppId(), topicName); + getHeartbeatTopicName(brokerId, pulsar.getConfiguration(), (topicVersion == TopicVersion.V2)); + LOG.info("[{}] Running healthCheck with topic={}", clientAppId, topicName); final String messageStr = UUID.randomUUID().toString(); final String subscriptionName = "healthCheck-" + messageStr; // create non-partitioned topic manually and close the previous reader if present. - return pulsar().getBrokerService().getTopic(topicName, true) + return pulsar.getBrokerService().getTopic(topicName, true) .thenCompose(topicOptional -> { if (!topicOptional.isPresent()) { LOG.error("[{}] Fail to run health check while get topic {}. because get null value.", - clientAppId(), topicName); + clientAppId, topicName); throw new RestException(Status.NOT_FOUND, String.format("Topic [%s] not found after create.", topicName)); } PulsarClient client; try { - client = pulsar().getClient(); + client = pulsar.getClient(); } catch (PulsarServerException e) { - LOG.error("[{}] Fail to run health check while get client.", clientAppId()); + LOG.error("[{}] Fail to run health check while get client.", clientAppId); throw new RestException(e); } CompletableFuture resultFuture = new CompletableFuture<>(); @@ -451,17 +461,18 @@ private CompletableFuture internalRunHealthCheck(TopicVersion topicVersion .startMessageId(MessageId.latest) .createAsync().exceptionally(createException -> { producer.closeAsync().exceptionally(ex -> { - LOG.error("[{}] Close producer fail while heath check.", clientAppId()); + LOG.error("[{}] Close producer fail while heath check.", clientAppId); return null; }); throw FutureUtil.wrapToCompletionException(createException); }).thenCompose(reader -> producer.sendAsync(messageStr) .thenCompose(__ -> FutureUtil.addTimeoutHandling( healthCheckRecursiveReadNext(reader, messageStr), - HEALTH_CHECK_READ_TIMEOUT, pulsar().getBrokerService().executor(), + HEALTH_CHECK_READ_TIMEOUT, pulsar.getBrokerService().executor(), () -> HEALTH_CHECK_TIMEOUT_EXCEPTION)) .whenComplete((__, ex) -> { - closeAndReCheck(producer, reader, topicOptional.get(), subscriptionName) + closeAndReCheck(producer, reader, topicOptional.get(), subscriptionName, + clientAppId) .whenComplete((unused, innerEx) -> { if (ex != null) { resultFuture.completeExceptionally(ex); @@ -479,6 +490,11 @@ HEALTH_CHECK_READ_TIMEOUT, pulsar().getBrokerService().executor(), }); } + private CompletableFuture closeAndReCheck(Producer producer, Reader reader, + Topic topic, String subscriptionName) { + return closeAndReCheck(producer, reader, topic, subscriptionName, clientAppId()); + } + /** * Close producer and reader and then to re-check if this operation is success. * @@ -491,8 +507,8 @@ HEALTH_CHECK_READ_TIMEOUT, pulsar().getBrokerService().executor(), * @param topic Topic * @param subscriptionName Subscription name */ - private CompletableFuture closeAndReCheck(Producer producer, Reader reader, - Topic topic, String subscriptionName) { + private static CompletableFuture closeAndReCheck(Producer producer, Reader reader, + Topic topic, String subscriptionName, String clientAppId) { // no matter exception or success, we still need to // close producer/reader CompletableFuture producerFuture = producer.closeAsync(); @@ -503,7 +519,7 @@ private CompletableFuture closeAndReCheck(Producer producer, Reade return FutureUtil.waitForAll(Collections.unmodifiableList(futures)) .exceptionally(closeException -> { if (readerFuture.isCompletedExceptionally()) { - LOG.error("[{}] Close reader fail while heath check.", clientAppId()); + LOG.error("[{}] Close reader fail while heath check.", clientAppId); Subscription subscription = topic.getSubscription(subscriptionName); // re-check subscription after reader close @@ -511,24 +527,24 @@ private CompletableFuture closeAndReCheck(Producer producer, Reade LOG.warn("[{}] Force delete subscription {} " + "when it still exists after the" + " reader is closed.", - clientAppId(), subscription); + clientAppId, subscription); subscription.deleteForcefully() .exceptionally(ex -> { LOG.error("[{}] Force delete subscription fail" + " while health check", - clientAppId(), ex); + clientAppId, ex); return null; }); } } else { // producer future fail. - LOG.error("[{}] Close producer fail while heath check.", clientAppId()); + LOG.error("[{}] Close producer fail while heath check.", clientAppId); } return null; }); } - private CompletableFuture healthCheckRecursiveReadNext(Reader reader, String content) { + private static CompletableFuture healthCheckRecursiveReadNext(Reader reader, String content) { return reader.readNextAsync() .thenCompose(msg -> { if (!Objects.equals(content, msg.getValue())) { 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 dd722dffcfbfc..c240c758dcda6 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 @@ -24,6 +24,7 @@ import static org.apache.bookkeeper.mledger.ManagedLedgerConfig.PROPERTY_SOURCE_TOPIC_KEY; import static org.apache.commons.collections4.CollectionUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.pulsar.broker.admin.impl.BrokersBase.internalRunHealthCheck; import static org.apache.pulsar.client.util.RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; import static org.apache.pulsar.client.util.RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX; import static org.apache.pulsar.common.naming.SystemTopicNames.isTransactionInternalName; @@ -157,6 +158,7 @@ import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; @@ -241,6 +243,7 @@ public class BrokerService implements Closeable { private AuthorizationService authorizationService; private final ScheduledExecutorService statsUpdater; + @Getter private final ScheduledExecutorService backlogQuotaChecker; @@ -346,6 +349,7 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.acceptorGroup = EventLoopUtil.newEventLoopGroup( pulsar.getConfiguration().getNumAcceptorThreads(), false, acceptorThreadFactory); this.workerGroup = eventLoopGroup; + this.statsUpdater = OrderedScheduler.newSchedulerBuilder() .name("pulsar-stats-updater") .numThreads(1) @@ -611,6 +615,7 @@ public void start() throws Exception { this.startStatsUpdater( serviceConfig.getStatsUpdateInitialDelayInSecs(), serviceConfig.getStatsUpdateFrequencyInSecs()); + this.initializeHealthChecker(); this.startInactivityMonitor(); this.startMessageExpiryMonitor(); this.startCompactionMonitor(); @@ -640,6 +645,24 @@ protected void startStatsUpdater(int statsUpdateInitialDelayInSecs, int statsUpd updateRates(); } + protected void initializeHealthChecker() { + ServiceConfiguration config = pulsar().getConfiguration(); + if (config.getHealthCheckMetricsUpdateTimeInSeconds() > 0) { + int interval = config.getHealthCheckMetricsUpdateTimeInSeconds(); + statsUpdater.scheduleAtFixedRate(this::checkHealth, + interval, interval, TimeUnit.SECONDS); + } + } + + public CompletableFuture checkHealth() { + return internalRunHealthCheck(TopicVersion.V2, pulsar(), null).thenAccept(__ -> { + this.pulsarStats.getBrokerOperabilityMetrics().recordHealthCheckStatusSuccess(); + }).exceptionally(ex -> { + this.pulsarStats.getBrokerOperabilityMetrics().recordHealthCheckStatusFail(); + return null; + }); + } + protected void startDeduplicationSnapshotMonitor() { // We do not know whether users will enable deduplication on namespace level/topic level or not, so keep this // scheduled task runs. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java index 3f991be8184ab..1855e1798b465 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java @@ -42,6 +42,7 @@ public class BrokerOperabilityMetrics implements AutoCloseable { private final LongAdder connectionTotalCreatedCount; private final LongAdder connectionTotalClosedCount; private final LongAdder connectionActive; + private volatile int healthCheckStatus; // 1=success, 0=failure, -1=unknown private final LongAdder connectionCreateSuccessCount; private final LongAdder connectionCreateFailCount; @@ -61,7 +62,7 @@ public BrokerOperabilityMetrics(PulsarService pulsar) { this.connectionTotalCreatedCount = new LongAdder(); this.connectionTotalClosedCount = new LongAdder(); this.connectionActive = new LongAdder(); - + this.healthCheckStatus = -1; this.connectionCreateSuccessCount = new LongAdder(); this.connectionCreateFailCount = new LongAdder(); @@ -103,6 +104,7 @@ private void generate() { reset(); metricsList.add(getTopicLoadMetrics()); metricsList.add(getConnectionMetrics()); + metricsList.add(getHealthMetrics()); } public Metrics generateConnectionMetrics() { @@ -119,6 +121,12 @@ Metrics getConnectionMetrics() { return rMetrics; } + Metrics getHealthMetrics() { + Metrics rMetrics = Metrics.create(getDimensionMap("broker_health")); + rMetrics.put("brk_health", healthCheckStatus); + return rMetrics; + } + Map getDimensionMap(String metricsName) { Map dimensionMap = new HashMap<>(); dimensionMap.put("broker", brokerName); @@ -179,4 +187,12 @@ public void recordConnectionCreateSuccess() { public void recordConnectionCreateFail() { this.connectionCreateFailCount.increment(); } + + public void recordHealthCheckStatusSuccess() { + this.healthCheckStatus = 1; + } + + public void recordHealthCheckStatusFail() { + this.healthCheckStatus = 0; + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java index 640cd2d37e399..36e741f8fa9cd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java @@ -1607,6 +1607,38 @@ public void testBrokerConnectionStats() throws Exception { assertEquals((long) map.get("brk_connection_create_fail_count"), 1); } + /** + * There is detailed info about this test. + * see: https://github.com/apache/pulsar/issues/10150#issuecomment-1112380074 + */ + @Test + public void testBrokerHealthCheckStatus() throws Exception { + + cleanup(); + conf.setSystemTopicEnabled(false); + conf.setTopicLevelPoliciesEnabled(false); + conf.setHealthCheckMetricsUpdateTimeInSeconds(60); + setup(); + BrokerService brokerService = this.pulsar.getBrokerService(); + + Map map = null; + + brokerService.checkHealth().get(); + brokerService.updateRates(); + Awaitility.await().until(() -> this.activeCount.get() == 1); + List metrics = brokerService.getTopicMetrics(); + System.out.println(metrics); + + for (int i = 0; i < metrics.size(); i++) { + if (metrics.get(i).getDimensions().containsValue("broker_health")) { + map = metrics.get(i).getMetrics(); + break; + } + } + assertNotNull(map); + assertEquals(map.get("brk_health"), 1); + } + @Test public void testPayloadCorruptionDetection() throws Exception { final String topicName = "persistent://prop/ns-abc/topic1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index a92f5a4acc208..fa073d3694b26 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -79,6 +79,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.manager.UnloadManager; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.service.AbstractTopic; +import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor; @@ -1789,6 +1790,20 @@ public void testBrokerConnection() throws Exception { compareBrokerConnectionStateCount(cm, 2.0); } + @Test + public void testBrokerHealthCheckMetric() throws Exception { + conf.setHealthCheckMetricsUpdateTimeInSeconds(60); + BrokerService brokerService = pulsar.getBrokerService(); + brokerService.checkHealth().get(); + brokerService.updateRates(); + ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); + String metricsStr = statsOut.toString(); + Multimap metrics = parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_health"); + compareBrokerConnectionStateCount(cm, 1); + } + private void compareBrokerConnectionStateCount(List cm, double count) { assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); @@ -1894,7 +1909,6 @@ public void testMetricsWithCache() throws Throwable { PrometheusMetricsGenerator prometheusMetricsGenerator = new PrometheusMetricsGenerator(pulsar, true, false, false, false, clock); - String previousMetrics = null; for (int a = 0; a < 4; a++) { ByteArrayOutputStream statsOut1 = new ByteArrayOutputStream(); @@ -1908,7 +1922,6 @@ public void testMetricsWithCache() throws Throwable { assertEquals(metricsStr1, metricsStr2); assertNotEquals(metricsStr1, previousMetrics); previousMetrics = metricsStr1; - // move time forward currentTimeMillis.addAndGet(TimeUnit.SECONDS.toMillis(2)); } From 806fdf86866813edbc8f6dea688df823c2889cc5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 7 Oct 2024 15:44:47 +0300 Subject: [PATCH 044/327] [improve][misc] Upgrade Jetty to 9.4.56.v20240826 (#23405) --- .../server/src/assemble/LICENSE.bin.txt | 38 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 16 ++++---- pom.xml | 2 +- 3 files changed, 28 insertions(+), 28 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 8c6e2cfa7159a..24eb6b8066df1 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -392,25 +392,25 @@ The Apache Software License, Version 2.0 - org.asynchttpclient-async-http-client-2.12.1.jar - org.asynchttpclient-async-http-client-netty-utils-2.12.1.jar * Jetty - - org.eclipse.jetty-jetty-client-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-continuation-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-http-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-io-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-proxy-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-security-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-server-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-servlet-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-servlets-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-util-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-util-ajax-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-api-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-client-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-common-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-server-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-servlet-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-alpn-server-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-client-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-continuation-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-http-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-io-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-proxy-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-security-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-server-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-servlet-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-servlets-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-util-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-util-ajax-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-api-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-client-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-common-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-server-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-servlet-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-alpn-server-9.4.56.v20240826.jar * SnakeYaml -- org.yaml-snakeyaml-2.0.jar * RocksDB - org.rocksdb-rocksdbjni-7.9.2.jar * Google Error Prone Annotations - com.google.errorprone-error_prone_annotations-2.24.0.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 6e0bacb2e8845..15b2a918b9ebc 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -402,14 +402,14 @@ The Apache Software License, Version 2.0 - async-http-client-2.12.1.jar - async-http-client-netty-utils-2.12.1.jar * Jetty - - jetty-client-9.4.54.v20240208.jar - - jetty-http-9.4.54.v20240208.jar - - jetty-io-9.4.54.v20240208.jar - - jetty-util-9.4.54.v20240208.jar - - javax-websocket-client-impl-9.4.54.v20240208.jar - - websocket-api-9.4.54.v20240208.jar - - websocket-client-9.4.54.v20240208.jar - - websocket-common-9.4.54.v20240208.jar + - jetty-client-9.4.56.v20240826.jar + - jetty-http-9.4.56.v20240826.jar + - jetty-io-9.4.56.v20240826.jar + - jetty-util-9.4.56.v20240826.jar + - javax-websocket-client-impl-9.4.56.v20240826.jar + - websocket-api-9.4.56.v20240826.jar + - websocket-client-9.4.56.v20240826.jar + - websocket-common-9.4.56.v20240826.jar * SnakeYaml -- snakeyaml-2.0.jar * Google Error Prone Annotations - error_prone_annotations-2.24.0.jar * Javassist -- javassist-3.25.0-GA.jar diff --git a/pom.xml b/pom.xml index c50357b840616..e0bce0442e158 100644 --- a/pom.xml +++ b/pom.xml @@ -148,7 +148,7 @@ flexible messaging model and an intuitive client API. 5.1.0 4.1.113.Final 0.0.24.Final - 9.4.54.v20240208 + 9.4.56.v20240826 2.5.2 2.42 1.10.50 From 5451921cd49dca03c541617c92ee8a3c83af9e50 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 7 Oct 2024 18:37:55 +0300 Subject: [PATCH 045/327] [improve] PIP-384: ManagedLedger interface decoupling (#23363) --- pip/pip-384.md | 158 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 158 insertions(+) create mode 100644 pip/pip-384.md diff --git a/pip/pip-384.md b/pip/pip-384.md new file mode 100644 index 0000000000000..ba02a147d857c --- /dev/null +++ b/pip/pip-384.md @@ -0,0 +1,158 @@ +# PIP-384: ManagedLedger interface decoupling + +## Background knowledge + +Apache Pulsar uses a component called ManagedLedger to handle persistent storage of messages. + +The ManagedLedger interfaces and implementation were initially tightly coupled, making it difficult to introduce alternative implementations or improve the architecture. +This PIP documents changes that have been made in the master branch for Pulsar 4.0. Pull Requests [#22891](https://github.com/apache/pulsar/pull/22891) and [#23311](https://github.com/apache/pulsar/pull/23311) have already been merged. +This work happened after lazy consensus on the dev mailing list based on the discussion thread ["Preparing for Pulsar 4.0: cleaning up the Managed Ledger interfaces"](https://lists.apache.org/thread/l5zjq0fb2dscys3rsn6kfl7505tbndlx). +There is one remaining PR [#23313](https://github.com/apache/pulsar/pull/23313) at the time of writing this document. +The goal of this PIP is to document the changes in this area for later reference. + +Key concepts: + +- **ManagedLedger**: A component that handles the persistent storage of messages in Pulsar. +- **BookKeeper**: The default storage system used by ManagedLedger. +- **ManagedLedgerStorage interface**: A factory for configuring and creating the `ManagedLedgerFactory` instance. [ManagedLedgerStorage.java source code](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java) +- **ManagedLedgerFactory interface**: Creates and manages ManagedLedger instances. [ManagedLedgerFactory.java source code](https://github.com/apache/pulsar/blob/master/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java) +- **ManagedLedger interface**: Handles the persistent storage of messages in Pulsar. [ManagedLedger.java source code](https://github.com/apache/pulsar/blob/master/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java) +- **ManagedCursor interface**: Handles the persistent storage of Pulsar subscriptions and related message acknowledgements. [ManagedCursor.java source code](https://github.com/apache/pulsar/blob/master/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java) + +## Motivation + +The current ManagedLedger implementation faces several challenges: + +1. **Tight coupling**: The interfaces are tightly coupled with their implementation, making it difficult to introduce alternative implementations. + +2. **Limited flexibility**: The current architecture doesn't allow for easy integration of different storage systems or optimizations. + +3. **Dependency on BookKeeper**: The ManagedLedger implementation is closely tied to BookKeeper, limiting options for alternative storage solutions. + +4. **Complexity**: The tight coupling increases the overall complexity of the system, making it harder to maintain, test and evolve. + +5. **Limited extensibility**: Introducing new features or optimizations often requires changes to both interfaces and implementations. + +## Goals + +### In Scope + +- Decouple ManagedLedger interfaces from their current implementation. +- Introduce a ReadOnlyManagedLedger interface. +- Decouple OpAddEntry and LedgerHandle from ManagedLedgerInterceptor. +- Enable support for multiple ManagedLedgerFactory instances. +- Decouple BookKeeper client from ManagedLedgerStorage. +- Improve overall architecture by reducing coupling between core Pulsar components and specific ManagedLedger implementations. +- Prepare the groundwork for alternative ManagedLedger implementations in Pulsar 4.0. + +### Out of Scope + +- Implementing alternative ManagedLedger storage backends. +- Changes to external APIs or behaviors. +- Comprehensive JavaDocs for the interfaces. + +## High Level Design + +1. **Decouple interfaces from implementations**: + - Move required methods from implementation classes to their respective interfaces. + - Update code to use interfaces instead of concrete implementations. + +2. **Introduce ReadOnlyManagedLedger interface**: + - Extract this interface to decouple from ReadOnlyManagedLedgerImpl. + - Adjust code to use the new interface where appropriate. + +3. **Decouple ManagedLedgerInterceptor**: + - Introduce AddEntryOperation and LastEntryHandle interfaces. + - Adjust ManagedLedgerInterceptor to use these new interfaces. + +4. **Enable multiple ManagedLedgerFactory instances**: + - Modify ManagedLedgerStorage interface to support multiple "storage classes". + - Implement BookkeeperManagedLedgerStorageClass for BookKeeper support. + - Update PulsarService and related classes to support multiple ManagedLedgerFactory instances. + - Add "storage class" to persistence policy part of the namespace level or topic level policies. + +5. **Decouple BookKeeper client**: + - Move BookKeeper client creation and management to BookkeeperManagedLedgerStorageClass. + - Update ManagedLedgerStorage interface to remove direct BookKeeper dependencies. + +## Detailed Design + +### Interface Decoupling + +1. Update ManagedLedger interface: + - Add methods from ManagedLedgerImpl to the interface. + - Remove dependencies on implementation-specific classes. + +2. Update ManagedLedgerFactory interface: + - Add necessary methods from ManagedLedgerFactoryImpl. + - Remove dependencies on implementation-specific classes. + +3. Update ManagedCursor interface: + - Add required methods from ManagedCursorImpl. + - Remove dependencies on implementation-specific classes. + +4. Introduce ReadOnlyManagedLedger interface: + - Extract methods specific to read-only operations. + - Update relevant code to use this interface where appropriate. + +5. Decouple ManagedLedgerInterceptor: + - Introduce AddEntryOperation interface for beforeAddEntry method. + - Introduce LastEntryHandle interface for onManagedLedgerLastLedgerInitialize method. + - Update ManagedLedgerInterceptor to use these new interfaces. + +### Multiple ManagedLedgerFactory Instances + +1. Update ManagedLedgerStorage interface: + - Add methods to support multiple storage classes. + - Introduce getManagedLedgerStorageClass method to retrieve specific storage implementations. + +2. Implement BookkeeperManagedLedgerStorageClass: + - Create a new class implementing ManagedLedgerStorageClass for BookKeeper. + - Move BookKeeper client creation and management to this class. + +3. Update PulsarService and related classes: + - Modify to support creation and management of multiple ManagedLedgerFactory instances. + - Update configuration to allow specifying different storage classes for different namespaces or topics. + +### BookKeeper Client Decoupling + +1. Update ManagedLedgerStorage interface: + - Remove direct dependencies on BookKeeper client. + - Introduce methods to interact with storage without exposing BookKeeper specifics. + +2. Implement BookkeeperManagedLedgerStorageClass: + - Encapsulate BookKeeper client creation and management. + - Implement storage operations using BookKeeper client. + +3. Update relevant code: + - Replace direct BookKeeper client usage with calls to ManagedLedgerStorage methods. + - Update configuration handling to support BookKeeper-specific settings through the new storage class. + +## Public-facing Changes + +### Configuration + +- Add new configuration option to specify default ManagedLedger "storage class" at broker level. + +### API Changes + +- No major changes to external APIs are planned. +- The only API change is to add `managedLedgerStorageClassName` to `PersistencePolicies` which can be used by a custom `ManagedLedgerStorage` to control the ManagedLedgerFactory instance that is used for a particular namespace or topic. + +## Backward & Forward Compatibility + +The changes are internal and don't affect external APIs or behaviors. +Backward compatibility is fully preserved in Apache Pulsar. + +## Security Considerations + +The decoupling of interfaces and implementation doesn't introduce new security concerns. + +## Links + +- Initial mailing List discussion thread: [Preparing for Pulsar 4.0: cleaning up the Managed Ledger interfaces](https://lists.apache.org/thread/l5zjq0fb2dscys3rsn6kfl7505tbndlx) + - Merged Pull Request #22891: [Replace dependencies on PositionImpl with Position interface](https://github.com/apache/pulsar/pull/22891) + - Merged Pull Request #23311: [Decouple ManagedLedger interfaces from the current implementation](https://github.com/apache/pulsar/pull/23311) + - Implementation Pull Request #23313: [Decouple Bookkeeper client from ManagedLedgerStorage and enable multiple ManagedLedgerFactory instances](https://github.com/apache/pulsar/pull/23313) +- Mailing List PIP discussion thread: https://lists.apache.org/thread/rtnktrj7tp5ppog0235t2mf9sxrdpfr8 +- Mailing List PIP voting thread: https://lists.apache.org/thread/4jj5dmk6jtpq05lcd6dxlkqpn7hov5gv \ No newline at end of file From 731ec8364f050e3db1532ec8316cf76109865e3d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 8 Oct 2024 04:48:09 +0300 Subject: [PATCH 046/327] [improve][broker][PIP-384] Decouple Bookkeeper client from ManagedLedgerStorage and enable multiple ManagedLedgerFactory instances (#23313) --- .../mledger/ManagedLedgerConfig.java | 4 +- .../broker/ManagedLedgerClientFactory.java | 50 ++++- .../apache/pulsar/broker/PulsarService.java | 28 ++- .../TransactionMetadataStoreService.java | 4 +- .../broker/admin/impl/NamespacesBase.java | 2 +- .../admin/impl/PersistentTopicsBase.java | 36 ++-- .../pulsar/broker/service/BrokerService.java | 87 ++++++--- .../pulsar/broker/service/ServerCnx.java | 95 ++++++---- .../service/persistent/PersistentTopic.java | 44 +++-- .../broker/stats/metrics/AbstractMetrics.java | 4 +- .../stats/metrics/ManagedLedgerMetrics.java | 3 +- .../PrometheusMetricsGenerator.java | 21 ++- .../BookkeeperManagedLedgerStorageClass.java | 42 +++++ .../broker/storage/ManagedLedgerStorage.java | 36 ++-- .../storage/ManagedLedgerStorageClass.java | 45 +++++ ...napshotSegmentAbortedTxnProcessorImpl.java | 9 +- .../impl/MLPendingAckStoreProvider.java | 175 +++++++++++------- .../broker/admin/AdminApiOffloadTest.java | 2 +- .../pulsar/broker/admin/AdminApiTest.java | 18 +- .../broker/admin/PersistentTopicsTest.java | 2 +- .../service/BrokerBkEnsemblesTests.java | 6 +- .../service/BrokerBookieIsolationTest.java | 6 +- .../broker/service/BrokerServiceTest.java | 7 +- ...sistentDispatcherFailoverConsumerTest.java | 6 +- .../PersistentTopicConcurrentTest.java | 2 +- .../service/PersistentTopicE2ETest.java | 4 +- .../broker/service/PersistentTopicTest.java | 15 +- .../broker/service/ReplicationTxnTest.java | 4 +- .../pulsar/broker/service/ReplicatorTest.java | 3 +- .../pulsar/broker/service/ServerCnxTest.java | 22 ++- .../service/TransactionMarkerDeleteTest.java | 2 +- .../stats/ManagedLedgerMetricsTest.java | 4 +- .../NonStartableTestPulsarService.java | 2 +- .../broker/testcontext/PulsarTestContext.java | 87 ++++++--- .../pulsar/broker/testcontext/SpyConfig.java | 3 + .../StartableTestPulsarService.java | 21 +++ .../TopicTransactionBufferRecoverTest.java | 2 +- .../transaction/TransactionProduceTest.java | 2 +- .../broker/transaction/TransactionTest.java | 2 +- .../pendingack/PendingAckPersistentTest.java | 17 +- .../client/api/OrphanPersistentTopicTest.java | 2 +- .../api/SimpleProducerConsumerTest.java | 39 +++- ...SubscriptionPauseOnAckStatPersistTest.java | 2 +- .../api/v1/V1_ProducerConsumerTest.java | 2 +- .../client/impl/SequenceIdWithErrorTest.java | 2 +- .../pulsar/compaction/CompactionTest.java | 4 +- .../policies/data/PersistencePolicies.java | 20 +- .../pulsar/admin/cli/CmdNamespaces.java | 8 +- .../pulsar/admin/cli/CmdTopicPolicies.java | 8 +- .../apache/pulsar/admin/cli/CmdTopics.java | 8 +- 50 files changed, 701 insertions(+), 318 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/BookkeeperManagedLedgerStorageClass.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorageClass.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 7b28990f35574..a1e1deb503e20 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -87,7 +87,9 @@ public class ManagedLedgerConfig { private int minimumBacklogEntriesForCaching = 1000; private int maxBacklogBetweenCursorsForCaching = 1000; private boolean triggerOffloadOnTopicLoad = false; - + @Getter + @Setter + private String storageClassName; @Getter @Setter private String shadowSourceName; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java index 9bbc2857863ff..737bc69bf24df 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java @@ -24,6 +24,8 @@ import io.netty.channel.EventLoopGroup; import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; +import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -39,16 +41,18 @@ import org.apache.bookkeeper.stats.StatsProvider; import org.apache.commons.configuration.Configuration; import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusMetricsProvider; +import org.apache.pulsar.broker.storage.BookkeeperManagedLedgerStorageClass; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ManagedLedgerClientFactory implements ManagedLedgerStorage { - private static final Logger log = LoggerFactory.getLogger(ManagedLedgerClientFactory.class); - + private static final String DEFAULT_STORAGE_CLASS_NAME = "bookkeeper"; + private BookkeeperManagedLedgerStorageClass defaultStorageClass; private ManagedLedgerFactory managedLedgerFactory; private BookKeeper defaultBkClient; private final AsyncCache @@ -119,20 +123,50 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata defaultBkClient.close(); throw e; } + + defaultStorageClass = new BookkeeperManagedLedgerStorageClass() { + @Override + public String getName() { + return DEFAULT_STORAGE_CLASS_NAME; + } + + @Override + public ManagedLedgerFactory getManagedLedgerFactory() { + return managedLedgerFactory; + } + + @Override + public StatsProvider getStatsProvider() { + return statsProvider; + } + + @Override + public BookKeeper getBookKeeperClient() { + return defaultBkClient; + } + }; } - public ManagedLedgerFactory getManagedLedgerFactory() { - return managedLedgerFactory; + @Override + public Collection getStorageClasses() { + return List.of(getDefaultStorageClass()); } - public BookKeeper getBookKeeperClient() { - return defaultBkClient; + @Override + public Optional getManagedLedgerStorageClass(String name) { + if (name == null || DEFAULT_STORAGE_CLASS_NAME.equals(name)) { + return Optional.of(getDefaultStorageClass()); + } else { + return Optional.empty(); + } } - public StatsProvider getStatsProvider() { - return statsProvider; + @Override + public ManagedLedgerStorageClass getDefaultStorageClass() { + return defaultStorageClass; } + @VisibleForTesting public Map getBkEnsemblePolicyToBookKeeperMap() { return bkEnsemblePolicyToBkClientMap.synchronous().asMap(); 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 6c768a078974f..dcc0e961275bd 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 @@ -124,7 +124,9 @@ import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; import org.apache.pulsar.broker.stats.prometheus.PulsarPrometheusMetricsServlet; +import org.apache.pulsar.broker.storage.BookkeeperManagedLedgerStorageClass; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; @@ -210,7 +212,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private static final int DEFAULT_MONOTONIC_CLOCK_GRANULARITY_MILLIS = 8; private final ServiceConfiguration config; private NamespaceService nsService = null; - private ManagedLedgerStorage managedLedgerClientFactory = null; + private ManagedLedgerStorage managedLedgerStorage = null; private LeaderElectionService leaderElectionService = null; private BrokerService brokerService = null; private WebService webService = null; @@ -606,13 +608,13 @@ public CompletableFuture closeAsync() { this.brokerService = null; } - if (this.managedLedgerClientFactory != null) { + if (this.managedLedgerStorage != null) { try { - this.managedLedgerClientFactory.close(); + this.managedLedgerStorage.close(); } catch (Exception e) { LOG.warn("ManagedLedgerClientFactory closing failed {}", e.getMessage()); } - this.managedLedgerClientFactory = null; + this.managedLedgerStorage = null; } if (bkClientFactory != null) { @@ -899,7 +901,7 @@ public void start() throws PulsarServerException { // Now we are ready to start services this.bkClientFactory = newBookKeeperClientFactory(); - managedLedgerClientFactory = newManagedLedgerClientFactory(); + managedLedgerStorage = newManagedLedgerStorage(); this.brokerService = newBrokerService(this); @@ -1122,7 +1124,7 @@ protected OrderedExecutor newOrderedExecutor() { } @VisibleForTesting - protected ManagedLedgerStorage newManagedLedgerClientFactory() throws Exception { + protected ManagedLedgerStorage newManagedLedgerStorage() throws Exception { return ManagedLedgerStorage.create( config, localMetadataStore, bkClientFactory, ioEventLoopGroup, openTelemetry.getOpenTelemetryService().getOpenTelemetry() @@ -1348,7 +1350,7 @@ private synchronized void startLoadBalancerTasks() { long resourceQuotaUpdateInterval = TimeUnit.MINUTES .toMillis(getConfiguration().getLoadBalancerResourceQuotaUpdateIntervalMinutes()); loadSheddingTask = new LoadSheddingTask(loadManager, loadManagerExecutor, - config, getManagedLedgerFactory()); + config, getDefaultManagedLedgerFactory()); loadSheddingTask.start(); loadResourceQuotaTask = loadManagerExecutor.scheduleAtFixedRate( new LoadResourceQuotaUpdaterTask(loadManager), resourceQuotaUpdateInterval, @@ -1535,11 +1537,17 @@ public WorkerService getWorkerService() throws UnsupportedOperationException { } public BookKeeper getBookKeeperClient() { - return getManagedLedgerClientFactory().getBookKeeperClient(); + ManagedLedgerStorageClass defaultStorageClass = getManagedLedgerStorage().getDefaultStorageClass(); + if (defaultStorageClass instanceof BookkeeperManagedLedgerStorageClass bkStorageClass) { + return bkStorageClass.getBookKeeperClient(); + } else { + // TODO: Refactor code to support other than default bookkeeper based storage class + throw new UnsupportedOperationException("BookKeeper client is not available"); + } } - public ManagedLedgerFactory getManagedLedgerFactory() { - return getManagedLedgerClientFactory().getManagedLedgerFactory(); + public ManagedLedgerFactory getDefaultManagedLedgerFactory() { + return getManagedLedgerStorage().getDefaultStorageClass().getManagedLedgerFactory(); } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/TransactionMetadataStoreService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/TransactionMetadataStoreService.java index c80580b02f19a..bd19a8e860255 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/TransactionMetadataStoreService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/TransactionMetadataStoreService.java @@ -227,7 +227,9 @@ public CompletableFuture handleTcClientConnect(TransactionCoordinatorID tc .setBatchedWriteMaxDelayInMillis(serviceConfiguration.getTransactionLogBatchedWriteMaxDelayInMillis()); return pulsarService.getBrokerService().getManagedLedgerConfig(getMLTransactionLogName(tcId)).thenCompose( - v -> transactionMetadataStoreProvider.openStore(tcId, pulsarService.getManagedLedgerFactory(), v, + v -> transactionMetadataStoreProvider.openStore(tcId, + pulsarService.getManagedLedgerStorage().getManagedLedgerStorageClass(v.getStorageClassName()) + .get().getManagedLedgerFactory(), v, timeoutTracker, recoverTracker, pulsarService.getConfig().getMaxActiveTransactionsPerCoordinator(), txnLogBufferedWriterConfig, brokerClientSharedTimer)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index 4d26fe2a4c35b..18c80d6bef4bf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -2540,7 +2540,7 @@ protected void internalScanOffloadedLedgers(OffloaderObjectsScannerUtils.Scanner String localClusterName = pulsar().getConfiguration().getClusterName(); OffloaderObjectsScannerUtils.scanOffloadedLedgers(managedLedgerOffloader, - localClusterName, pulsar().getManagedLedgerFactory(), sink); + localClusterName, pulsar().getDefaultManagedLedgerFactory(), sink); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 8860c9bb06d4d..6070093cc3585 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1405,19 +1405,27 @@ protected void internalGetManagedLedgerInfoForNonPartitionedTopic(AsyncResponse validateTopicOperationAsync(topicName, TopicOperation.GET_STATS) .thenAccept(__ -> { String managedLedger = topicName.getPersistenceNamingEncoding(); - pulsar().getManagedLedgerFactory() - .asyncGetManagedLedgerInfo(managedLedger, new ManagedLedgerInfoCallback() { - @Override - public void getInfoComplete(ManagedLedgerInfo info, Object ctx) { - asyncResponse.resume((StreamingOutput) output -> { - objectWriter().writeValue(output, info); + pulsar().getBrokerService().getManagedLedgerFactoryForTopic(topicName) + .thenAccept(managedLedgerFactory -> { + managedLedgerFactory.asyncGetManagedLedgerInfo(managedLedger, + new ManagedLedgerInfoCallback() { + @Override + public void getInfoComplete(ManagedLedgerInfo info, Object ctx) { + asyncResponse.resume((StreamingOutput) output -> { + objectWriter().writeValue(output, info); + }); + } + + @Override + public void getInfoFailed(ManagedLedgerException exception, Object ctx) { + asyncResponse.resume(exception); + } + }, null); + }) + .exceptionally(ex -> { + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; }); - } - @Override - public void getInfoFailed(ManagedLedgerException exception, Object ctx) { - asyncResponse.resume(exception); - } - }, null); }).exceptionally(ex -> { log.error("[{}] Failed to get managed info for {}", clientAppId(), topicName, ex); resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3174,7 +3182,9 @@ protected CompletableFuture internalGetBacklogAsync try { PersistentOfflineTopicStats estimateOfflineTopicStats = offlineTopicBacklog.estimateUnloadedTopicBacklog( - pulsar().getManagedLedgerFactory(), + pulsar().getBrokerService() + .getManagedLedgerFactoryForTopic(topicName, + config.getStorageClassName()), topicName); pulsar().getBrokerService() .cacheOfflineTopicStats(topicName, estimateOfflineTopicStats); 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 c240c758dcda6..ed0cdf18b47ca 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 @@ -64,6 +64,7 @@ import java.util.Set; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; @@ -136,6 +137,8 @@ import org.apache.pulsar.broker.stats.ClusterReplicationMetrics; import org.apache.pulsar.broker.stats.prometheus.metrics.ObserverGauge; import org.apache.pulsar.broker.stats.prometheus.metrics.Summary; +import org.apache.pulsar.broker.storage.ManagedLedgerStorage; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.broker.validator.BindAddressValidator; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -216,7 +219,7 @@ public class BrokerService implements Closeable { .register(); private final PulsarService pulsar; - private final ManagedLedgerFactory managedLedgerFactory; + private final ManagedLedgerStorage managedLedgerStorage; private final Map>> topics = new ConcurrentHashMap<>(); @@ -335,7 +338,7 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.brokerPublishRateLimiter = new PublishRateLimiterImpl(pulsar.getMonotonicSnapshotClock()); this.preciseTopicPublishRateLimitingEnable = pulsar.getConfiguration().isPreciseTopicPublishRateLimiterEnable(); - this.managedLedgerFactory = pulsar.getManagedLedgerFactory(); + this.managedLedgerStorage = pulsar.getManagedLedgerStorage(); this.keepAliveIntervalSeconds = pulsar.getConfiguration().getKeepAliveIntervalSeconds(); this.pendingTopicLoadingQueue = Queues.newConcurrentLinkedQueue(); this.pulsarStats = new PulsarStats(pulsar); @@ -1241,23 +1244,51 @@ private CompletableFuture deleteTopicInternal(String topic, boolean forceD return; } CompletableFuture mlConfigFuture = getManagedLedgerConfig(topicName); - managedLedgerFactory.asyncDelete(tn.getPersistenceNamingEncoding(), - mlConfigFuture, new DeleteLedgerCallback() { - @Override - public void deleteLedgerComplete(Object ctx) { - future.complete(null); - } + mlConfigFuture.thenAccept(config -> { + getManagedLedgerFactoryForTopic(topicName, config.getStorageClassName()) + .asyncDelete(tn.getPersistenceNamingEncoding(), + mlConfigFuture, new DeleteLedgerCallback() { + @Override + public void deleteLedgerComplete(Object ctx) { + future.complete(null); + } - @Override - public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { - future.completeExceptionally(exception); - } - }, null); + @Override + public void deleteLedgerFailed(ManagedLedgerException exception, + Object ctx) { + future.completeExceptionally(exception); + } + }, null); + }).exceptionally(ex1 -> { + log.error("Failed to get managed ledger config for topic {}", topic, ex1); + future.completeExceptionally(ex1); + return null; + }); }); return future; } + public CompletableFuture getManagedLedgerFactoryForTopic(TopicName topicName) { + return getManagedLedgerConfig(topicName) + .thenApply(config -> { + String storageClassName = config.getStorageClassName(); + return getManagedLedgerFactoryForTopic(topicName, storageClassName); + }); + } + + public ManagedLedgerFactory getManagedLedgerFactoryForTopic(TopicName topicName, String storageClassName) { + Optional managedLedgerStorageClass = + managedLedgerStorage.getManagedLedgerStorageClass(storageClassName); + if (!managedLedgerStorageClass.isPresent()) { + throw new CompletionException(new ManagedLedgerException( + "ManagedLedgerStorageClass " + storageClassName + " not found for topic " + topicName)); + } + return managedLedgerStorageClass + .get() + .getManagedLedgerFactory(); + } + public void deleteTopicAuthenticationWithRetry(String topic, CompletableFuture future, int count) { if (count == 0) { log.error("The number of retries has exhausted for topic {}", topic); @@ -1624,14 +1655,17 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S @VisibleForTesting protected CompletableFuture> fetchTopicPropertiesAsync(TopicName topicName) { if (!topicName.isPartitioned()) { - return managedLedgerFactory.getManagedLedgerPropertiesAsync(topicName.getPersistenceNamingEncoding()); + return getManagedLedgerFactoryForTopic(topicName).thenCompose( + managedLedgerFactory -> managedLedgerFactory.getManagedLedgerPropertiesAsync( + topicName.getPersistenceNamingEncoding())); } else { TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); return fetchPartitionedTopicMetadataAsync(partitionedTopicName) .thenCompose(metadata -> { if (metadata.partitions == PartitionedTopicMetadata.NON_PARTITIONED) { - return managedLedgerFactory.getManagedLedgerPropertiesAsync( - topicName.getPersistenceNamingEncoding()); + return getManagedLedgerFactoryForTopic(topicName).thenCompose( + managedLedgerFactory -> managedLedgerFactory.getManagedLedgerPropertiesAsync( + topicName.getPersistenceNamingEncoding())); } else { // Check if the partitioned topic is a ShadowTopic if (MapUtils.getString(metadata.properties, PROPERTY_SOURCE_TOPIC_KEY) != null) { @@ -1756,6 +1790,8 @@ private void createPersistentTopic(final String topic, boolean createIfMissing, topicEventsDispatcher.notifyOnCompletion(loadFuture, topic, TopicEvent.LOAD); // Once we have the configuration, we can proceed with the async open operation + ManagedLedgerFactory managedLedgerFactory = + getManagedLedgerFactoryForTopic(topicName, managedLedgerConfig.getStorageClassName()); managedLedgerFactory.asyncOpen(topicName.getPersistenceNamingEncoding(), managedLedgerConfig, new OpenLedgerCallback() { @Override @@ -1918,6 +1954,7 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T managedLedgerConfig.setEnsembleSize(persistencePolicies.getBookkeeperEnsemble()); managedLedgerConfig.setWriteQuorumSize(persistencePolicies.getBookkeeperWriteQuorum()); managedLedgerConfig.setAckQuorumSize(persistencePolicies.getBookkeeperAckQuorum()); + managedLedgerConfig.setStorageClassName(persistencePolicies.getManagedLedgerStorageClassName()); if (serviceConfig.isStrictBookieAffinityEnabled()) { managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyClassName( @@ -2745,25 +2782,29 @@ private void updateConfigurationAndRegisterListeners() { }); }); + + ManagedLedgerFactory defaultManagedLedgerFactory = + managedLedgerStorage.getDefaultStorageClass().getManagedLedgerFactory(); + // add listener to notify broker managedLedgerCacheSizeMB dynamic config registerConfigurationListener("managedLedgerCacheSizeMB", (managedLedgerCacheSizeMB) -> { - managedLedgerFactory.getEntryCacheManager() + defaultManagedLedgerFactory.getEntryCacheManager() .updateCacheSizeAndThreshold(((int) managedLedgerCacheSizeMB) * 1024L * 1024L); }); // add listener to notify broker managedLedgerCacheEvictionWatermark dynamic config registerConfigurationListener( "managedLedgerCacheEvictionWatermark", (cacheEvictionWatermark) -> { - managedLedgerFactory.getEntryCacheManager() - .updateCacheEvictionWatermark((double) cacheEvictionWatermark); - }); + defaultManagedLedgerFactory.getEntryCacheManager() + .updateCacheEvictionWatermark((double) cacheEvictionWatermark); + }); // add listener to notify broker managedLedgerCacheEvictionTimeThresholdMillis dynamic config registerConfigurationListener( "managedLedgerCacheEvictionTimeThresholdMillis", (cacheEvictionTimeThresholdMills) -> { - managedLedgerFactory.updateCacheEvictionTimeThreshold(MILLISECONDS - .toNanos((long) cacheEvictionTimeThresholdMills)); - }); + defaultManagedLedgerFactory.updateCacheEvictionTimeThreshold(MILLISECONDS + .toNanos((long) cacheEvictionTimeThresholdMills)); + }); // add listener to update message-dispatch-rate in msg for topic 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 aedd68d416fe7..37b431e833983 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 @@ -2902,7 +2902,8 @@ protected void handleEndTxnOnPartition(CommandEndTxnOnPartition command) { log.debug("[{}] handleEndTxnOnPartition txnId: [{}], txnAction: [{}]", topic, txnID, txnAction); } - CompletableFuture> topicFuture = service.getTopicIfExists(TopicName.get(topic).toString()); + TopicName topicName = TopicName.get(topic); + CompletableFuture> topicFuture = service.getTopicIfExists(topicName.toString()); topicFuture.thenAcceptAsync(optionalTopic -> { if (optionalTopic.isPresent()) { // we only accept superuser because this endpoint is reserved for tc to broker communication @@ -2928,24 +2929,29 @@ protected void handleEndTxnOnPartition(CommandEndTxnOnPartition command) { txnID.getLeastSigBits(), txnID.getMostSigBits())); }); } else { - getBrokerService().getManagedLedgerFactory() - .asyncExists(TopicName.get(topic).getPersistenceNamingEncoding()) - .thenAccept((b) -> { - if (b) { - log.error("handleEndTxnOnPartition fail ! The topic {} does not exist in broker, " - + "txnId: [{}], txnAction: [{}]", topic, - txnID, TxnAction.valueOf(txnAction)); - writeAndFlush(Commands.newEndTxnOnPartitionResponse(requestId, - ServerError.ServiceNotReady, - "The topic " + topic + " does not exist in broker.", - txnID.getLeastSigBits(), txnID.getMostSigBits())); - } else { - log.warn("handleEndTxnOnPartition fail ! The topic {} has not been created, " - + "txnId: [{}], txnAction: [{}]", - topic, txnID, TxnAction.valueOf(txnAction)); - writeAndFlush(Commands.newEndTxnOnPartitionResponse(requestId, - txnID.getLeastSigBits(), txnID.getMostSigBits())); - } + getBrokerService().getManagedLedgerFactoryForTopic(topicName) + .thenCompose(managedLedgerFactory -> { + return managedLedgerFactory.asyncExists(topicName.getPersistenceNamingEncoding()) + .thenAccept((b) -> { + if (b) { + log.error( + "handleEndTxnOnPartition fail ! The topic {} does not exist in " + + "broker, " + + "txnId: [{}], txnAction: [{}]", topic, + txnID, TxnAction.valueOf(txnAction)); + writeAndFlush(Commands.newEndTxnOnPartitionResponse(requestId, + ServerError.ServiceNotReady, + "The topic " + topic + " does not exist in broker.", + txnID.getLeastSigBits(), txnID.getMostSigBits())); + } else { + log.warn( + "handleEndTxnOnPartition fail ! The topic {} has not been created, " + + "txnId: [{}], txnAction: [{}]", + topic, txnID, TxnAction.valueOf(txnAction)); + writeAndFlush(Commands.newEndTxnOnPartitionResponse(requestId, + txnID.getLeastSigBits(), txnID.getMostSigBits())); + } + }); }).exceptionally(e -> { log.error("handleEndTxnOnPartition fail ! topic {}, " + "txnId: [{}], txnAction: [{}]", topic, txnID, @@ -2954,7 +2960,8 @@ protected void handleEndTxnOnPartition(CommandEndTxnOnPartition command) { requestId, ServerError.ServiceNotReady, e.getMessage(), txnID.getLeastSigBits(), txnID.getMostSigBits())); return null; - }); + + }); } }, ctx.executor()).exceptionally(e -> { log.error("handleEndTxnOnPartition fail ! topic {}, " @@ -2984,7 +2991,8 @@ protected void handleEndTxnOnSubscription(CommandEndTxnOnSubscription command) { new TxnID(txnidMostBits, txnidLeastBits), txnAction); } - CompletableFuture> topicFuture = service.getTopicIfExists(TopicName.get(topic).toString()); + TopicName topicName = TopicName.get(topic); + CompletableFuture> topicFuture = service.getTopicIfExists(topicName.toString()); topicFuture.thenAcceptAsync(optionalTopic -> { if (optionalTopic.isPresent()) { Subscription subscription = optionalTopic.get().getSubscription(subName); @@ -3019,24 +3027,31 @@ protected void handleEndTxnOnSubscription(CommandEndTxnOnSubscription command) { Commands.newEndTxnOnSubscriptionResponse(requestId, txnidLeastBits, txnidMostBits)); }); } else { - getBrokerService().getManagedLedgerFactory() - .asyncExists(TopicName.get(topic).getPersistenceNamingEncoding()) - .thenAccept((b) -> { - if (b) { - log.error("handleEndTxnOnSubscription fail! The topic {} does not exist in broker, " - + "subscription: {}, txnId: [{}], txnAction: [{}]", topic, subName, - txnID, TxnAction.valueOf(txnAction)); - writeAndFlush(Commands.newEndTxnOnSubscriptionResponse( - requestId, txnID.getLeastSigBits(), txnID.getMostSigBits(), - ServerError.ServiceNotReady, - "The topic " + topic + " does not exist in broker.")); - } else { - log.warn("handleEndTxnOnSubscription fail ! The topic {} has not been created, " - + "subscription: {} txnId: [{}], txnAction: [{}]", - topic, subName, txnID, TxnAction.valueOf(txnAction)); - writeAndFlush(Commands.newEndTxnOnSubscriptionResponse(requestId, - txnID.getLeastSigBits(), txnID.getMostSigBits())); - } + getBrokerService().getManagedLedgerFactoryForTopic(topicName) + .thenCompose(managedLedgerFactory -> { + return managedLedgerFactory.asyncExists(topicName.getPersistenceNamingEncoding()) + .thenAccept((b) -> { + if (b) { + log.error( + "handleEndTxnOnSubscription fail! The topic {} does not exist in " + + "broker, " + + "subscription: {}, txnId: [{}], txnAction: [{}]", topic, + subName, + txnID, TxnAction.valueOf(txnAction)); + writeAndFlush(Commands.newEndTxnOnSubscriptionResponse( + requestId, txnID.getLeastSigBits(), txnID.getMostSigBits(), + ServerError.ServiceNotReady, + "The topic " + topic + " does not exist in broker.")); + } else { + log.warn( + "handleEndTxnOnSubscription fail ! The topic {} has not been " + + "created, " + + "subscription: {} txnId: [{}], txnAction: [{}]", + topic, subName, txnID, TxnAction.valueOf(txnAction)); + writeAndFlush(Commands.newEndTxnOnSubscriptionResponse(requestId, + txnID.getLeastSigBits(), txnID.getMostSigBits())); + } + }); }).exceptionally(e -> { log.error("handleEndTxnOnSubscription fail ! topic {}, subscription: {}" + "txnId: [{}], txnAction: [{}]", topic, subName, @@ -3045,7 +3060,7 @@ protected void handleEndTxnOnSubscription(CommandEndTxnOnSubscription command) { requestId, txnID.getLeastSigBits(), txnID.getMostSigBits(), ServerError.ServiceNotReady, e.getMessage())); return null; - }); + }); } }, ctx.executor()).exceptionally(e -> { log.error("handleEndTxnOnSubscription fail ! topic: {}, subscription: {}" 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 f8581cfc79985..3cce175660e70 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 @@ -74,12 +74,14 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerAlreadyClosedException; import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerFencedException; import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerTerminatedException; import org.apache.bookkeeper.mledger.ManagedLedgerException.MetadataNotFoundException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.PositionFactory; @@ -1232,26 +1234,34 @@ public CompletableFuture unsubscribe(String subscriptionName) { .getTransactionPendingAckStoreSuffix(topic, Codec.encode(subscriptionName))); if (brokerService.pulsar().getConfiguration().isTransactionCoordinatorEnabled()) { - getBrokerService().getManagedLedgerFactory().asyncDelete(tn.getPersistenceNamingEncoding(), - getBrokerService().getManagedLedgerConfig(tn), - new AsyncCallbacks.DeleteLedgerCallback() { - @Override - public void deleteLedgerComplete(Object ctx) { - asyncDeleteCursorWithClearDelayedMessage(subscriptionName, unsubscribeFuture); - } - - @Override - public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { - if (exception instanceof MetadataNotFoundException) { + CompletableFuture managedLedgerConfig = getBrokerService().getManagedLedgerConfig(tn); + managedLedgerConfig.thenAccept(config -> { + ManagedLedgerFactory managedLedgerFactory = + getBrokerService().getManagedLedgerFactoryForTopic(tn, config.getStorageClassName()); + managedLedgerFactory.asyncDelete(tn.getPersistenceNamingEncoding(), + managedLedgerConfig, + new AsyncCallbacks.DeleteLedgerCallback() { + @Override + public void deleteLedgerComplete(Object ctx) { asyncDeleteCursorWithClearDelayedMessage(subscriptionName, unsubscribeFuture); - return; } - unsubscribeFuture.completeExceptionally(exception); - log.error("[{}][{}] Error deleting subscription pending ack store", - topic, subscriptionName, exception); - } - }, null); + @Override + public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { + if (exception instanceof MetadataNotFoundException) { + asyncDeleteCursorWithClearDelayedMessage(subscriptionName, unsubscribeFuture); + return; + } + + unsubscribeFuture.completeExceptionally(exception); + log.error("[{}][{}] Error deleting subscription pending ack store", + topic, subscriptionName, exception); + } + }, null); + }).exceptionally(ex -> { + unsubscribeFuture.completeExceptionally(ex); + return null; + }); } else { asyncDeleteCursorWithClearDelayedMessage(subscriptionName, unsubscribeFuture); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java index 489d37dd0a307..114f962cb81d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java @@ -131,7 +131,7 @@ protected Metrics createMetrics(Map dimensionMap) { * @return */ protected ManagedLedgerFactoryMXBean getManagedLedgerCacheStats() { - return pulsar.getManagedLedgerFactory().getCacheStats(); + return pulsar.getDefaultManagedLedgerFactory().getCacheStats(); } /** @@ -140,7 +140,7 @@ protected ManagedLedgerFactoryMXBean getManagedLedgerCacheStats() { * @return */ protected Map getManagedLedgers() { - return pulsar.getManagedLedgerFactory().getManagedLedgers(); + return pulsar.getDefaultManagedLedgerFactory().getManagedLedgers(); } protected String getLocalClusterName() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java index 52c69265c2f1f..925fcb28b7a03 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java @@ -52,8 +52,7 @@ public ManagedLedgerMetrics(PulsarService pulsar) { this.metricsCollection = new ArrayList<>(); this.ledgersByDimensionMap = new HashMap<>(); this.tempAggregatedMetricsMap = new HashMap<>(); - this.statsPeriodSeconds = pulsar.getManagedLedgerFactory() - .getConfig().getStatsPeriodSeconds(); + this.statsPeriodSeconds = pulsar.getDefaultManagedLedgerFactory().getConfig().getStatsPeriodSeconds(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index 6b4d08c359d42..8c3cb39c925d7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -55,6 +55,8 @@ import org.apache.pulsar.broker.stats.metrics.ManagedCursorMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerCacheMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerMetrics; +import org.apache.pulsar.broker.storage.BookkeeperManagedLedgerStorageClass; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.SimpleTextOutputStream; @@ -485,12 +487,14 @@ private static SimpleTextOutputStream writeNameReplacingBrkPrefix(SimpleTextOutp } private static void generateManagedLedgerBookieClientMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { - StatsProvider statsProvider = pulsar.getManagedLedgerClientFactory().getStatsProvider(); - if (statsProvider instanceof NullStatsProvider) { - return; - } + ManagedLedgerStorageClass defaultStorageClass = pulsar.getManagedLedgerStorage().getDefaultStorageClass(); + if (defaultStorageClass instanceof BookkeeperManagedLedgerStorageClass bkStorageClass) { + StatsProvider statsProvider = bkStorageClass.getStatsProvider(); + if (statsProvider instanceof NullStatsProvider) { + return; + } - try (Writer writer = new OutputStreamWriter(new BufferedOutputStream(new OutputStream() { + try (Writer writer = new OutputStreamWriter(new BufferedOutputStream(new OutputStream() { @Override public void write(int b) throws IOException { stream.writeByte(b); @@ -501,9 +505,10 @@ public void write(byte b[], int off, int len) throws IOException { stream.write(b, off, len); } }), StandardCharsets.UTF_8)) { - statsProvider.writeAllMetrics(writer); - } catch (IOException e) { - log.error("Failed to write managed ledger bookie client metrics", e); + statsProvider.writeAllMetrics(writer); + } catch (IOException e) { + log.error("Failed to write managed ledger bookie client metrics", e); + } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/BookkeeperManagedLedgerStorageClass.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/BookkeeperManagedLedgerStorageClass.java new file mode 100644 index 0000000000000..1f05cde72a5b5 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/BookkeeperManagedLedgerStorageClass.java @@ -0,0 +1,42 @@ +/* + * 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.storage; + +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.stats.StatsProvider; + +/** + * ManagedLedgerStorageClass represents a configured instance of ManagedLedgerFactory for managed ledgers. + * This instance is backed by a bookkeeper storage. + */ +public interface BookkeeperManagedLedgerStorageClass extends ManagedLedgerStorageClass { + /** + * Return the bookkeeper client instance used by this instance. + * + * @return the bookkeeper client. + */ + BookKeeper getBookKeeperClient(); + + /** + * Return the stats provider to expose the stats of the storage implementation. + * + * @return the stats provider. + */ + StatsProvider getStatsProvider(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java index 944d2badf75f2..720798123e7b4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java @@ -21,9 +21,8 @@ import io.netty.channel.EventLoopGroup; import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; -import org.apache.bookkeeper.client.BookKeeper; -import org.apache.bookkeeper.mledger.ManagedLedgerFactory; -import org.apache.bookkeeper.stats.StatsProvider; +import java.util.Collection; +import java.util.Optional; import org.apache.pulsar.broker.BookKeeperClientFactory; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.common.classification.InterfaceAudience.Private; @@ -33,6 +32,12 @@ /** * Storage to access {@link org.apache.bookkeeper.mledger.ManagedLedger}s. + *

+ * The interface provides the abstraction to access the storage layer for managed ledgers. + * The interface supports multiple storage classes, each with its own configuration. The default + * implementation supports a single instance of {@link BookkeeperManagedLedgerStorageClass}. + * Implementations can provide multiple storage classes. The default storage class is used + * for topics unless it is overridden by the persistency policy at topic or namespace level. */ @Private @Unstable @@ -52,25 +57,25 @@ void initialize(ServiceConfiguration conf, OpenTelemetry openTelemetry) throws Exception; /** - * Return the factory to create {@link ManagedLedgerFactory}. - * - * @return the factory to create {@link ManagedLedgerFactory}. + * Get all configured storage class instances. + * @return all configured storage class instances */ - ManagedLedgerFactory getManagedLedgerFactory(); + Collection getStorageClasses(); /** - * Return the stats provider to expose the stats of the storage implementation. - * - * @return the stats provider. + * Get the default storage class. + * @return default storage class */ - StatsProvider getStatsProvider(); + default ManagedLedgerStorageClass getDefaultStorageClass() { + return getStorageClasses().stream().findFirst().get(); + } /** - * Return the default bookkeeper client. - * - * @return the default bookkeeper client. + * Lookup a storage class by name. + * @param name storage class name + * @return storage class instance, or empty if not found */ - BookKeeper getBookKeeperClient(); + Optional getManagedLedgerStorageClass(String name); /** * Close the storage. @@ -97,5 +102,4 @@ static ManagedLedgerStorage create(ServiceConfiguration conf, storage.initialize(conf, metadataStore, bkProvider, eventLoopGroup, openTelemetry); return storage; } - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorageClass.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorageClass.java new file mode 100644 index 0000000000000..8cbe5c3b411e5 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorageClass.java @@ -0,0 +1,45 @@ +/* + * 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.storage; + +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; +import org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; + +/** + * ManagedLedgerStorageClass represents a configured instance of ManagedLedgerFactory for managed ledgers. + * The {@link ManagedLedgerStorage} can hold multiple storage classes, and each storage class can have its own + * configuration. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface ManagedLedgerStorageClass { + /** + * Return the name of the storage class. + * + * @return the name of the storage class. + */ + String getName(); + /** + * Return the factory to create {@link ManagedLedgerFactory}. + * + * @return the factory to create {@link ManagedLedgerFactory}. + */ + ManagedLedgerFactory getManagedLedgerFactory(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java index 88a3968b7b430..f2ff5d519d8c0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java @@ -311,8 +311,13 @@ public String toString() { SnapshotSegmentAbortedTxnProcessorImpl.this.topic.getName()); } }; - topic.getBrokerService().getPulsar().getManagedLedgerFactory().asyncOpenReadOnlyManagedLedger( - topicName.getPersistenceNamingEncoding(), callback, topic.getManagedLedger().getConfig(), null); + topic.getBrokerService().getManagedLedgerFactoryForTopic(topicName).thenAccept(managedLedgerFactory -> + managedLedgerFactory.asyncOpenReadOnlyManagedLedger(topicName.getPersistenceNamingEncoding(), + callback, topic.getManagedLedger().getConfig(), null)) + .exceptionally(e -> { + future.completeExceptionally(e); + return null; + }); return wait(future, "open read only ml for " + topicName); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java index 6fc61d423ce85..12f761bb4df5e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java @@ -25,9 +25,11 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.exception.pendingack.TransactionPendingAckException; @@ -84,7 +86,8 @@ public CompletableFuture newPendingAckStore(PersistentSubscript } PersistentTopic originPersistentTopic = (PersistentTopic) subscription.getTopic(); - PulsarService pulsarService = originPersistentTopic.getBrokerService().getPulsar(); + BrokerService brokerService = originPersistentTopic.getBrokerService(); + PulsarService pulsarService = brokerService.getPulsar(); final Timer brokerClientSharedTimer = pulsarService.getBrokerClientSharedTimer(); @@ -103,93 +106,127 @@ public CompletableFuture newPendingAckStore(PersistentSubscript String pendingAckTopicName = MLPendingAckStore .getTransactionPendingAckStoreSuffix(originPersistentTopic.getName(), subscription.getName()); - originPersistentTopic.getBrokerService().getManagedLedgerFactory() - .asyncExists(TopicName.get(pendingAckTopicName) - .getPersistenceNamingEncoding()).thenAccept(exist -> { - TopicName topicName; - if (exist) { - topicName = TopicName.get(pendingAckTopicName); - } else { - topicName = TopicName.get(originPersistentTopic.getName()); - } - originPersistentTopic.getBrokerService() - .getManagedLedgerConfig(topicName).thenAccept(config -> { - config.setCreateIfMissing(true); - originPersistentTopic.getBrokerService().getManagedLedgerFactory() - .asyncOpen(TopicName.get(pendingAckTopicName).getPersistenceNamingEncoding(), - config, new AsyncCallbacks.OpenLedgerCallback() { - @Override - public void openLedgerComplete(ManagedLedger ledger, Object ctx) { - ledger.asyncOpenCursor( - MLPendingAckStore.getTransactionPendingAckStoreCursorName(), - InitialPosition.Earliest, new AsyncCallbacks.OpenCursorCallback() { - @Override - public void openCursorComplete(ManagedCursor cursor, Object ctx) { - pendingAckStoreFuture.complete(new MLPendingAckStore(ledger, - cursor, - subscription.getCursor(), - originPersistentTopic - .getBrokerService() - .getPulsar() - .getConfiguration() - .getTransactionPendingAckLogIndexMinLag(), - txnLogBufferedWriterConfig, - brokerClientSharedTimer, bufferedWriterMetrics, - originPersistentTopic - .getBrokerService() - .getPulsar() - .getOrderedExecutor() - .chooseThread())); - if (log.isDebugEnabled()) { - log.debug("{},{} open MLPendingAckStore cursor success", - originPersistentTopic.getName(), - subscription.getName()); - } - } - - @Override - public void openCursorFailed(ManagedLedgerException exception, - Object ctx) { - log.error("{},{} open MLPendingAckStore cursor failed." - , originPersistentTopic.getName(), - subscription.getName(), exception); - pendingAckStoreFuture.completeExceptionally(exception); - } - }, null); - } - - @Override - public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { - log.error("{}, {} open MLPendingAckStore managedLedger failed." - , originPersistentTopic.getName(), subscription.getName(), exception); - pendingAckStoreFuture.completeExceptionally(exception); - } - }, () -> CompletableFuture.completedFuture(true), null); + TopicName pendingAckTopicNameObject = TopicName.get(pendingAckTopicName); + brokerService.getManagedLedgerFactoryForTopic(pendingAckTopicNameObject) + .thenAccept(managedLedgerFactory -> { + managedLedgerFactory.asyncExists(pendingAckTopicNameObject + .getPersistenceNamingEncoding()).thenAccept(exist -> { + TopicName topicName; + if (exist) { + topicName = pendingAckTopicNameObject; + } else { + topicName = TopicName.get(originPersistentTopic.getName()); + } + brokerService.getManagedLedgerConfig(topicName).thenAccept(config -> { + internalNewPendingAckStore(subscription, config, brokerService, topicName, + pendingAckTopicNameObject, pendingAckStoreFuture, txnLogBufferedWriterConfig, + brokerClientSharedTimer, originPersistentTopic); + }).exceptionally(e -> { + Throwable t = FutureUtil.unwrapCompletionException(e); + log.error("[{}] [{}] Failed to get managedLedger config when init pending ack " + + "store!", + originPersistentTopic, subscription, t); + pendingAckStoreFuture.completeExceptionally(t); + return null; + + }); }).exceptionally(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); - log.error("[{}] [{}] Failed to get managedLedger config when init pending ack store!", + log.error("[{}] [{}] Failed to check the pending ack topic exist when init pending ack store!", originPersistentTopic, subscription, t); pendingAckStoreFuture.completeExceptionally(t); return null; - }); }).exceptionally(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); - log.error("[{}] [{}] Failed to check the pending ack topic exist when init pending ack store!", - originPersistentTopic, subscription, t); + log.error("[{}] [{}] Failed to get managedLedger config when init pending ack store!", + pendingAckTopicNameObject, subscription, t); pendingAckStoreFuture.completeExceptionally(t); return null; }); return pendingAckStoreFuture; } + private static void internalNewPendingAckStore(PersistentSubscription subscription, ManagedLedgerConfig config, + BrokerService brokerService, TopicName topicName, + TopicName pendingAckTopicNameObject, + CompletableFuture pendingAckStoreFuture, + TxnLogBufferedWriterConfig txnLogBufferedWriterConfig, + Timer brokerClientSharedTimer, + PersistentTopic originPersistentTopic) { + config.setCreateIfMissing(true); + brokerService + .getManagedLedgerFactoryForTopic(topicName, config.getStorageClassName()) + .asyncOpen(pendingAckTopicNameObject.getPersistenceNamingEncoding(), + config, new AsyncCallbacks.OpenLedgerCallback() { + @Override + public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + ledger.asyncOpenCursor( + MLPendingAckStore.getTransactionPendingAckStoreCursorName(), + InitialPosition.Earliest, + new AsyncCallbacks.OpenCursorCallback() { + @Override + public void openCursorComplete(ManagedCursor cursor, + Object ctx) { + pendingAckStoreFuture.complete( + new MLPendingAckStore(ledger, + cursor, + subscription.getCursor(), + brokerService + .getPulsar() + .getConfiguration() + .getTransactionPendingAckLogIndexMinLag(), + txnLogBufferedWriterConfig, + brokerClientSharedTimer, + bufferedWriterMetrics, + brokerService + .getPulsar() + .getOrderedExecutor() + .chooseThread())); + if (log.isDebugEnabled()) { + log.debug( + "{},{} open MLPendingAckStore cursor " + + "success", + originPersistentTopic.getName(), + subscription.getName()); + } + } + + @Override + public void openCursorFailed( + ManagedLedgerException exception, + Object ctx) { + log.error( + "{},{} open MLPendingAckStore cursor " + + "failed." + , originPersistentTopic.getName(), + subscription.getName(), exception); + pendingAckStoreFuture.completeExceptionally( + exception); + } + }, null); + } + + @Override + public void openLedgerFailed(ManagedLedgerException exception, + Object ctx) { + log.error("{}, {} open MLPendingAckStore managedLedger failed." + , originPersistentTopic.getName(), subscription.getName(), + exception); + pendingAckStoreFuture.completeExceptionally(exception); + } + }, () -> CompletableFuture.completedFuture(true), null); + } + @Override public CompletableFuture checkInitializedBefore(PersistentSubscription subscription) { PersistentTopic originPersistentTopic = (PersistentTopic) subscription.getTopic(); String pendingAckTopicName = MLPendingAckStore .getTransactionPendingAckStoreSuffix(originPersistentTopic.getName(), subscription.getName()); - return originPersistentTopic.getBrokerService().getManagedLedgerFactory() - .asyncExists(TopicName.get(pendingAckTopicName).getPersistenceNamingEncoding()); + TopicName topicName = TopicName.get(pendingAckTopicName); + return originPersistentTopic.getBrokerService().getManagedLedgerFactoryForTopic(topicName) + .thenCompose(managedLedgerFactory -> managedLedgerFactory.asyncExists( + topicName.getPersistenceNamingEncoding())); } private static class MLTxnPendingAckLogBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{ diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java index 1ea29c9d431bd..9aa2dcc700c9b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java @@ -135,7 +135,7 @@ private void testOffload(String topicName, String mlName) throws Exception { } } - ManagedLedgerInfo info = pulsar.getManagedLedgerFactory().getManagedLedgerInfo(mlName); + ManagedLedgerInfo info = pulsar.getDefaultManagedLedgerFactory().getManagedLedgerInfo(mlName); assertEquals(info.ledgers.size(), 2); assertEquals(admin.topics().offloadStatus(topicName).getStatus(), LongRunningProcessStatus.Status.NOT_RUN); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 4a1dbface2c63..26da4116d09cb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -588,24 +588,24 @@ public void testUpdateDynamicCacheConfigurationWithZkWatch() throws Exception { // wait config to be updated Awaitility.await().until(() -> { - return pulsar.getManagedLedgerFactory().getEntryCacheManager().getMaxSize() == 1 * 1024L * 1024L - && pulsar.getManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark() == 0.8 - && pulsar.getManagedLedgerFactory().getCacheEvictionTimeThreshold() == TimeUnit.MILLISECONDS + return pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getMaxSize() == 1 * 1024L * 1024L + && pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark() == 0.8 + && pulsar.getDefaultManagedLedgerFactory().getCacheEvictionTimeThreshold() == TimeUnit.MILLISECONDS .toNanos(2000); }); // verify value is updated - assertEquals(pulsar.getManagedLedgerFactory().getEntryCacheManager().getMaxSize(), 1 * 1024L * 1024L); - assertEquals(pulsar.getManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark(), 0.8); - assertEquals(pulsar.getManagedLedgerFactory().getCacheEvictionTimeThreshold(), TimeUnit.MILLISECONDS + assertEquals(pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getMaxSize(), 1 * 1024L * 1024L); + assertEquals(pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark(), 0.8); + assertEquals(pulsar.getDefaultManagedLedgerFactory().getCacheEvictionTimeThreshold(), TimeUnit.MILLISECONDS .toNanos(2000)); restartBroker(); // verify value again - assertEquals(pulsar.getManagedLedgerFactory().getEntryCacheManager().getMaxSize(), 1 * 1024L * 1024L); - assertEquals(pulsar.getManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark(), 0.8); - assertEquals(pulsar.getManagedLedgerFactory().getCacheEvictionTimeThreshold(), TimeUnit.MILLISECONDS + assertEquals(pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getMaxSize(), 1 * 1024L * 1024L); + assertEquals(pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark(), 0.8); + assertEquals(pulsar.getDefaultManagedLedgerFactory().getCacheEvictionTimeThreshold(), TimeUnit.MILLISECONDS .toNanos(2000)); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index 18fd3dd1c8bb3..aae2f7b8830e8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -708,7 +708,7 @@ public void testUpdatePartitionedTopicHavingNonPartitionTopicWithPartitionSuffix // partitioned topic to more than 10. final String nonPartitionTopicName2 = "special-topic-partition-10"; final String partitionedTopicName = "special-topic"; - pulsar.getBrokerService().getManagedLedgerFactory() + pulsar.getDefaultManagedLedgerFactory() .open(TopicName.get(nonPartitionTopicName2).getPersistenceNamingEncoding()); doAnswer(invocation -> { persistentTopics.namespaceName = NamespaceName.get("tenant", "namespace"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java index 82892ad353aa1..68a52c4b4c381 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java @@ -127,7 +127,7 @@ public void testCrashBrokerWithoutCursorLedgerLeak() throws Exception { consumer.close(); producer.close(); pulsar.getBrokerService().removeTopicFromCache(topic); - ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); field.setAccessible(true); @SuppressWarnings("unchecked") @@ -250,7 +250,7 @@ public void testSkipCorruptDataLedger() throws Exception { // clean managed-ledger and recreate topic to clean any data from the cache producer.close(); pulsar.getBrokerService().removeTopicFromCache(topic); - ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); field.setAccessible(true); @SuppressWarnings("unchecked") @@ -399,7 +399,7 @@ public void testTruncateCorruptDataLedger() throws Exception { @Test public void testDeleteLedgerFactoryCorruptLedger() throws Exception { - ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open("test"); // bookkeeper client diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java index d7272fcffa964..be6f7c9143742 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java @@ -274,7 +274,7 @@ public void testBookieIsolation() throws Exception { assertAffinityBookies(ledgerManager, ml4.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = - (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); + (ManagedLedgerClientFactory) pulsarService.getManagedLedgerStorage(); Map bkPlacementPolicyToBkClientMap = mlFactory .getBkEnsemblePolicyToBookKeeperMap(); @@ -588,7 +588,7 @@ public void testStrictBookieIsolation() throws Exception { assertAffinityBookies(ledgerManager, ml4.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = - (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); + (ManagedLedgerClientFactory) pulsarService.getManagedLedgerStorage(); Map bkPlacementPolicyToBkClientMap = mlFactory .getBkEnsemblePolicyToBookKeeperMap(); @@ -751,7 +751,7 @@ public void testBookieIsolationWithSecondaryGroup() throws Exception { assertAffinityBookies(ledgerManager, ml3.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = - (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); + (ManagedLedgerClientFactory) pulsarService.getManagedLedgerStorage(); Map bkPlacementPolicyToBkClientMap = mlFactory .getBkEnsemblePolicyToBookKeeperMap(); 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 17209c83c13ea..e05bb836a3ce6 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 @@ -76,8 +76,8 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.bookkeeper.mledger.impl.NonAppendableLedgerOffloader; +import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.http.HttpResponse; import org.apache.http.client.methods.HttpGet; import org.apache.http.impl.client.CloseableHttpClient; @@ -1451,7 +1451,7 @@ public void testLedgerOpenFailureShouldNotHaveDeadLock() throws Exception { ledgerField.setAccessible(true); @SuppressWarnings("unchecked") ConcurrentHashMap> ledgers = (ConcurrentHashMap>) ledgerField - .get(pulsar.getManagedLedgerFactory()); + .get(pulsar.getDefaultManagedLedgerFactory()); CompletableFuture future = new CompletableFuture<>(); future.completeExceptionally(new ManagedLedgerException("ledger opening failed")); ledgers.put(namespace + "/persistent/deadLockTestTopic", future); @@ -1517,8 +1517,7 @@ public void testStuckTopicUnloading() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); - ManagedLedgerFactoryImpl mlFactory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerClientFactory() - .getManagedLedgerFactory(); + ManagedLedgerFactoryImpl mlFactory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); Field ledgersField = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); ledgersField.setAccessible(true); ConcurrentHashMap> ledgers = (ConcurrentHashMap>) ledgersField diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java index 000ea7af91525..69f3e2e4d3917 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java @@ -61,6 +61,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.ServiceConfiguration; @@ -184,10 +185,11 @@ void setupMLAsyncCallbackMocks() { doReturn("mockCursor").when(cursorMock).getName(); // call openLedgerComplete with ledgerMock on ML factory asyncOpen + ManagedLedgerFactory managedLedgerFactory = pulsarTestContext.getDefaultManagedLedgerFactory(); doAnswer(invocationOnMock -> { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -196,7 +198,7 @@ void setupMLAsyncCallbackMocks() { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]) .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java index 85e0887465db2..f75a32567473d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java @@ -92,7 +92,7 @@ public void setup(Method m) throws Exception { cursorMock = ledger.openCursor("c1"); ledgerMock = ledger; mlFactoryMock = factory; - doReturn(mlFactoryMock).when(pulsar).getManagedLedgerFactory(); + doReturn(mlFactoryMock).when(pulsar).getDefaultManagedLedgerFactory(); brokerService = spyWithClassAndConstructorArgs(BrokerService.class, pulsar, eventLoopGroup); doReturn(brokerService).when(pulsar).getBrokerService(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java index 36e741f8fa9cd..2896c13af0093 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java @@ -596,7 +596,7 @@ public void testUnloadNamespace() throws Exception { pulsarClient.close(); assertNotNull(pulsar.getBrokerService().getTopicReference(topic)); - assertTrue(((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers() + assertTrue(((ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory()).getManagedLedgers() .containsKey(topicName.getPersistenceNamingEncoding())); admin.namespaces().unload("prop/ns-abc"); @@ -613,7 +613,7 @@ public void testUnloadNamespace() throws Exception { } // ML should have been closed as well - assertFalse(((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers() + assertFalse(((ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory()).getManagedLedgers() .containsKey(topicName.getPersistenceNamingEncoding())); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index 81c12df4f3918..1e96da737dd51 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -90,6 +90,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; @@ -169,6 +170,7 @@ public class PersistentTopicTest extends MockedBookKeeperTestCase { private BrokerService brokerService; private EventLoopGroup eventLoopGroup; + private ManagedLedgerFactory managedLedgerFactory; @BeforeMethod(alwaysRun = true) public void setup() throws Exception { @@ -190,13 +192,14 @@ public void setup() throws Exception { .build(); brokerService = pulsarTestContext.getBrokerService(); + managedLedgerFactory = pulsarTestContext.getDefaultManagedLedgerFactory(); doAnswer(invocationOnMock -> CompletableFuture.completedFuture(null)) - .when(pulsarTestContext.getManagedLedgerFactory()).getManagedLedgerPropertiesAsync(any()); + .when(managedLedgerFactory).getManagedLedgerPropertiesAsync(any()); doAnswer(invocation -> { DeleteLedgerCallback deleteLedgerCallback = invocation.getArgument(1); deleteLedgerCallback.deleteLedgerComplete(null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()).asyncDelete(any(), any(), any()); + }).when(managedLedgerFactory).asyncDelete(any(), any(), any()); // Mock serviceCnx. serverCnx = spyWithClassAndConstructorArgsRecordingInvocations(ServerCnx.class, pulsarTestContext.getPulsarService()); @@ -247,7 +250,7 @@ public void testCreateTopic() { doAnswer(invocationOnMock -> { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(anyString(), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -273,7 +276,7 @@ public void testCreateTopicMLFailure() { .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null)).start(); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(anyString(), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -1395,7 +1398,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { doAnswer(invocationOnMock -> { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -1404,7 +1407,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]) .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java index 3caf4a1f2398c..bd4a0889c730f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java @@ -187,14 +187,14 @@ public void testTxnLogNotBeReplicated() throws Exception { for (int i = 0; i < txnLogPartitions; i++) { TopicName txnLog = TopicName.get(TopicDomain.persistent.value(), NamespaceName.SYSTEM_NAMESPACE, TRANSACTION_LOG_PREFIX + i); - assertNotNull(pulsar1.getManagedLedgerFactory() + assertNotNull(pulsar1.getDefaultManagedLedgerFactory() .getManagedLedgerInfo(txnLog.getPersistenceNamingEncoding())); assertFalse(broker1.getTopics().containsKey(txnLog.toString())); } // __transaction_pending_ack: it only uses ML, will not create topic. TopicName pendingAck = TopicName.get( MLPendingAckStore.getTransactionPendingAckStoreSuffix(topic, subscription)); - assertNotNull(pulsar1.getManagedLedgerFactory() + assertNotNull(pulsar1.getDefaultManagedLedgerFactory() .getManagedLedgerInfo(pendingAck.getPersistenceNamingEncoding())); assertFalse(broker1.getTopics().containsKey(pendingAck.toString())); // __transaction_buffer_snapshot. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index aac7a85f477c5..2420ed58bed27 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -1419,8 +1419,7 @@ public void testCleanupTopic() throws Exception { config1.setTopicLoadTimeoutSeconds(topicLoadTimeoutSeconds); config2.setTopicLoadTimeoutSeconds(topicLoadTimeoutSeconds); - ManagedLedgerFactoryImpl mlFactory = (ManagedLedgerFactoryImpl) pulsar1.getManagedLedgerClientFactory() - .getManagedLedgerFactory(); + ManagedLedgerFactoryImpl mlFactory = (ManagedLedgerFactoryImpl) pulsar1.getDefaultManagedLedgerFactory(); Field ledgersField = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); ledgersField.setAccessible(true); ConcurrentHashMap> ledgers = (ConcurrentHashMap>) ledgersField diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 42b52d901e32f..9a85995ab771f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -83,6 +83,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.lang3.mutable.MutableInt; @@ -202,6 +203,7 @@ public class ServerCnxTest { private ManagedLedger ledgerMock; private ManagedCursor cursorMock; private ConcurrentHashSet channelsStoppedAnswerHealthCheck = new ConcurrentHashSet<>(); + private ManagedLedgerFactory managedLedgerFactory; @BeforeMethod(alwaysRun = true) @@ -218,7 +220,7 @@ public void setup() throws Exception { .spyByDefault() .build(); pulsar = pulsarTestContext.getPulsarService(); - + managedLedgerFactory = pulsarTestContext.getDefaultManagedLedgerFactory(); brokerService = pulsarTestContext.getBrokerService(); namespaceService = pulsar.getNamespaceService(); @@ -2043,7 +2045,7 @@ public void testCreateProducerTimeout() throws Exception { () -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2098,7 +2100,7 @@ public void testCreateProducerTimeoutThenCreateSameNamedProducerShouldFail() thr () -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2165,7 +2167,7 @@ public void testCreateProducerMultipleTimeouts() throws Exception { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2244,7 +2246,7 @@ public void testCreateProducerBookieTimeout() throws Exception { () -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2316,7 +2318,7 @@ public void testSubscribeTimeout() throws Exception { null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2391,7 +2393,7 @@ public void testSubscribeBookieTimeout() throws Exception { () -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2400,7 +2402,7 @@ public void testSubscribeBookieTimeout() throws Exception { openTopicFail.complete(() -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]) .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2926,7 +2928,7 @@ private void setupMLAsyncCallbackMocks() { Thread.sleep(300); ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2937,7 +2939,7 @@ private void setupMLAsyncCallbackMocks() { .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null)).start(); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java index 7e8454f6c7eef..fc10d315cb14a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java @@ -70,7 +70,7 @@ protected void cleanup() throws Exception { @Test public void testMarkerDeleteTimes() throws Exception { ManagedLedgerImpl managedLedger = - spy((ManagedLedgerImpl) getPulsarServiceList().get(0).getManagedLedgerFactory().open("test")); + spy((ManagedLedgerImpl) getPulsarServiceList().get(0).getDefaultManagedLedgerFactory().open("test")); PersistentTopic topic = mock(PersistentTopic.class); BrokerService brokerService = mock(BrokerService.class); PulsarService pulsarService = mock(PulsarService.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java index d0fd384ba78fb..d72e8f75427bd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java @@ -105,7 +105,7 @@ public void testManagedLedgerMetrics() throws Exception { producer.send(message.getBytes()); } - var managedLedgerFactory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + var managedLedgerFactory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); for (Entry ledger : managedLedgerFactory.getManagedLedgers().entrySet()) { ManagedLedgerMBeanImpl stats = (ManagedLedgerMBeanImpl) ledger.getValue().getStats(); stats.refreshStats(1, TimeUnit.SECONDS); @@ -205,7 +205,7 @@ public void testTransactionTopic() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setMaxEntriesPerLedger(2); MLTransactionLogImpl mlTransactionLog = new MLTransactionLogImpl(TransactionCoordinatorID.get(0), - pulsar.getManagedLedgerFactory(), managedLedgerConfig, txnLogBufferedWriterConfig, + pulsar.getDefaultManagedLedgerFactory(), managedLedgerConfig, txnLogBufferedWriterConfig, transactionTimer, DISABLED_BUFFERED_WRITER_METRICS); mlTransactionLog.initialize().get(2, TimeUnit.SECONDS); ManagedLedgerMetrics metrics = new ManagedLedgerMetrics(pulsar); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/NonStartableTestPulsarService.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/NonStartableTestPulsarService.java index 7860b0708e35e..70e386c68aa26 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/NonStartableTestPulsarService.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/NonStartableTestPulsarService.java @@ -71,7 +71,7 @@ public NonStartableTestPulsarService(SpyConfig spyConfig, ServiceConfiguration c super(spyConfig, config, localMetadataStore, configurationMetadataStore, compactionServiceFactory, brokerInterceptor, bookKeeperClientFactory, null); setPulsarResources(pulsarResources); - setManagedLedgerClientFactory(managedLedgerClientFactory); + setManagedLedgerStorage(managedLedgerClientFactory); try { setBrokerService(brokerServiceCustomizer.apply( spyConfig.getBrokerService().spy(TestBrokerService.class, this, getIoEventLoopGroup()))); 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 3d79a17a90f50..cdb047079bfcd 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 @@ -28,6 +28,7 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.function.Consumer; @@ -56,7 +57,9 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.ServerCnx; import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil; +import org.apache.pulsar.broker.storage.BookkeeperManagedLedgerStorageClass; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.common.util.GracefulExecutorServicesShutdown; import org.apache.pulsar.common.util.PortManager; import org.apache.pulsar.compaction.CompactionServiceFactory; @@ -136,7 +139,7 @@ public class PulsarTestContext implements AutoCloseable { private final OrderedExecutor executor; - private final ManagedLedgerStorage managedLedgerClientFactory; + private final ManagedLedgerStorage managedLedgerStorage; private final PulsarService pulsarService; @@ -167,8 +170,12 @@ public class PulsarTestContext implements AutoCloseable { private final boolean enableOpenTelemetry; private final InMemoryMetricReader openTelemetryMetricReader; - public ManagedLedgerFactory getManagedLedgerFactory() { - return managedLedgerClientFactory.getManagedLedgerFactory(); + public ManagedLedgerStorage getManagedLedgerStorage() { + return managedLedgerStorage; + } + + public ManagedLedgerFactory getDefaultManagedLedgerFactory() { + return getManagedLedgerStorage().getDefaultStorageClass().getManagedLedgerFactory(); } public PulsarMockBookKeeper getMockBookKeeper() { @@ -524,8 +531,8 @@ public Builder useTestPulsarResources(MetadataStore metadataStore) { */ public Builder managedLedgerClients(BookKeeper bookKeeperClient, ManagedLedgerFactory managedLedgerFactory) { - return managedLedgerClientFactory( - PulsarTestContext.createManagedLedgerClientFactory(bookKeeperClient, managedLedgerFactory)); + return managedLedgerStorage( + PulsarTestContext.createManagedLedgerStorage(bookKeeperClient, managedLedgerFactory)); } /** @@ -569,6 +576,9 @@ public final PulsarTestContext build() { if (configOverrideCustomizer != null) { configOverrideCustomizer.accept(super.config); } + if (super.managedLedgerStorage != null && !MockUtil.isMock(super.managedLedgerStorage)) { + super.managedLedgerStorage = spyConfig.getManagedLedgerStorage().spy(super.managedLedgerStorage); + } initializeCommonPulsarServices(spyConfig); initializePulsarServices(spyConfig, this); if (pulsarServiceCustomizer != null) { @@ -622,7 +632,7 @@ protected void handlePreallocatePorts(ServiceConfiguration config) { } private void initializeCommonPulsarServices(SpyConfig spyConfig) { - if (super.bookKeeperClient == null && super.managedLedgerClientFactory == null) { + if (super.bookKeeperClient == null && super.managedLedgerStorage == null) { if (super.executor == null) { OrderedExecutor createdExecutor = OrderedExecutor.newBuilder().numThreads(1) .name(PulsarTestContext.class.getSimpleName() + "-executor").build(); @@ -645,8 +655,11 @@ private void initializeCommonPulsarServices(SpyConfig spyConfig) { }); bookKeeperClient(mockBookKeeper); } - if (super.bookKeeperClient == null && super.managedLedgerClientFactory != null) { - bookKeeperClient(super.managedLedgerClientFactory.getBookKeeperClient()); + if (super.bookKeeperClient == null && super.managedLedgerStorage != null) { + bookKeeperClient(super.managedLedgerStorage.getStorageClasses().stream() + .filter(BookkeeperManagedLedgerStorageClass.class::isInstance) + .map(BookkeeperManagedLedgerStorageClass.class::cast) + .map(BookkeeperManagedLedgerStorageClass::getBookKeeperClient).findFirst().get()); } if (super.localMetadataStore == null || super.configurationMetadataStore == null) { if (super.mockZooKeeper != null) { @@ -725,8 +738,8 @@ static class StartableCustomBuilder extends AbstractCustomBuilder { } @Override - public Builder managedLedgerClientFactory(ManagedLedgerStorage managedLedgerClientFactory) { - throw new IllegalStateException("Cannot set managedLedgerClientFactory when startable."); + public Builder managedLedgerStorage(ManagedLedgerStorage managedLedgerStorage) { + throw new IllegalStateException("Cannot set managedLedgerStorage when startable."); } @Override @@ -788,10 +801,12 @@ static class NonStartableCustomBuilder extends AbstractCustomBuilder { @Override protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { - if (builder.managedLedgerClientFactory == null) { + if (builder.managedLedgerStorage == null) { ManagedLedgerFactory mlFactoryMock = Mockito.mock(ManagedLedgerFactory.class); - managedLedgerClientFactory( - PulsarTestContext.createManagedLedgerClientFactory(builder.bookKeeperClient, mlFactoryMock)); + managedLedgerStorage( + spyConfig.getManagedLedgerStorage() + .spy(PulsarTestContext.createManagedLedgerStorage(builder.bookKeeperClient, + mlFactoryMock))); } if (builder.pulsarResources == null) { SpyConfig.SpyType spyConfigPulsarResources = spyConfig.getPulsarResources(); @@ -825,7 +840,7 @@ protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { builder.configurationMetadataStore, compactionServiceFactory, builder.brokerInterceptor, bookKeeperClientFactory, builder.pulsarResources, - builder.managedLedgerClientFactory, builder.brokerServiceCustomizer); + builder.managedLedgerStorage, builder.brokerServiceCustomizer); if (compactionServiceFactory != null) { compactionServiceFactory.initialize(pulsarService); } @@ -838,10 +853,31 @@ protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { } @NotNull - private static ManagedLedgerStorage createManagedLedgerClientFactory(BookKeeper bookKeeperClient, - ManagedLedgerFactory managedLedgerFactory) { - return new ManagedLedgerStorage() { + private static ManagedLedgerStorage createManagedLedgerStorage(BookKeeper bookKeeperClient, + ManagedLedgerFactory managedLedgerFactory) { + BookkeeperManagedLedgerStorageClass managedLedgerStorageClass = + new BookkeeperManagedLedgerStorageClass() { + @Override + public String getName() { + return "bookkeeper"; + } + + @Override + public ManagedLedgerFactory getManagedLedgerFactory() { + return managedLedgerFactory; + } + + @Override + public StatsProvider getStatsProvider() { + return new NullStatsProvider(); + } + @Override + public BookKeeper getBookKeeperClient() { + return bookKeeperClient; + } + }; + return new ManagedLedgerStorage() { @Override public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadataStore, BookKeeperClientFactory bookkeeperProvider, EventLoopGroup eventLoopGroup, @@ -849,18 +885,17 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata } @Override - public ManagedLedgerFactory getManagedLedgerFactory() { - return managedLedgerFactory; - } - - @Override - public StatsProvider getStatsProvider() { - return new NullStatsProvider(); + public Collection getStorageClasses() { + return List.of(managedLedgerStorageClass); } @Override - public BookKeeper getBookKeeperClient() { - return bookKeeperClient; + public Optional getManagedLedgerStorageClass(String name) { + if (name == null || name.equals("bookkeeper")) { + return Optional.of(managedLedgerStorageClass); + } else { + return Optional.empty(); + } } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/SpyConfig.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/SpyConfig.java index 64789d1f0d487..285eb1bba6d6e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/SpyConfig.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/SpyConfig.java @@ -112,6 +112,8 @@ public T spy(Class clazz, Object... args) { */ private final SpyType namespaceService; + private final SpyType managedLedgerStorage; + /** * Create a builder for SpyConfig with no spies by default. * @@ -141,5 +143,6 @@ public static void configureDefaults(Builder spyConfigBuilder, SpyType defaultSp spyConfigBuilder.compactor(defaultSpyType); spyConfigBuilder.compactedServiceFactory(defaultSpyType); spyConfigBuilder.namespaceService(defaultSpyType); + spyConfigBuilder.managedLedgerStorage(defaultSpyType); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/StartableTestPulsarService.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/StartableTestPulsarService.java index a0774414492dc..d82cd69c83dd0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/StartableTestPulsarService.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/StartableTestPulsarService.java @@ -23,6 +23,7 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import lombok.SneakyThrows; import org.apache.pulsar.broker.BookKeeperClientFactory; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -30,6 +31,7 @@ import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.storage.ManagedLedgerStorage; import org.apache.pulsar.compaction.CompactionServiceFactory; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -62,4 +64,23 @@ protected BrokerService newBrokerService(PulsarService pulsar) throws Exception public Supplier getNamespaceServiceProvider() throws PulsarServerException { return () -> spyConfig.getNamespaceService().spy(NamespaceService.class, this); } + + @SneakyThrows + @Override + public ManagedLedgerStorage getManagedLedgerStorage() { + // support adding spy to managedLedgerStorage in beforePulsarStart method + if (super.getManagedLedgerStorage() == null) { + setManagedLedgerStorage(createManagedLedgerStorageSpy()); + } + return super.getManagedLedgerStorage(); + } + + @Override + protected ManagedLedgerStorage newManagedLedgerStorage() throws Exception { + return getManagedLedgerStorage(); + } + + private ManagedLedgerStorage createManagedLedgerStorageSpy() throws Exception { + return spyConfig.getManagedLedgerStorage().spy(super.newManagedLedgerStorage()); + } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index f21e11b980209..14cc813a17ddd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -806,7 +806,7 @@ public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Ob // } }; - pulsarService.getManagedLedgerFactory() + pulsarService.getDefaultManagedLedgerFactory() .asyncOpenReadOnlyManagedLedger(snapshotSegmentTopicName.getPersistenceNamingEncoding(), callback, brokerService.getManagedLedgerConfig(snapshotSegmentTopicName).get(),null); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java index 14b1d563c11ec..3d7ab902bf494 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java @@ -247,7 +247,7 @@ private ReadOnlyCursor getOriginTopicCursor(String topic, int partition) { if (partition >= 0) { topic = TopicName.get(topic).toString() + TopicName.PARTITIONED_TOPIC_SUFFIX + partition; } - return getPulsarServiceList().get(0).getManagedLedgerFactory().openReadOnlyCursor( + return getPulsarServiceList().get(0).getDefaultManagedLedgerFactory().openReadOnlyCursor( TopicName.get(topic).getPersistenceNamingEncoding(), PositionFactory.EARLIEST, new ManagedLedgerConfig()); } catch (Exception e) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 5480b1a21d5a0..35c9048ebb554 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -510,7 +510,7 @@ public void testSubscriptionRecreateTopic() admin.topics().createNonPartitionedTopic(topic); PulsarService pulsarService = super.getPulsarServiceList().get(0); pulsarService.getBrokerService().getTopics().clear(); - ManagedLedgerFactory managedLedgerFactory = pulsarService.getBrokerService().getManagedLedgerFactory(); + ManagedLedgerFactory managedLedgerFactory = pulsarService.getDefaultManagedLedgerFactory(); Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); field.setAccessible(true); ConcurrentHashMap> ledgers = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index 72aa078d5da1e..fc6a10e385a54 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -23,7 +23,7 @@ import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -45,6 +45,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.BKException; @@ -952,8 +953,14 @@ public void testGetManagedLegerConfigFailThenUnload() throws Exception { assertNotNull(persistentTopic); BrokerService brokerService = spy(persistentTopic.getBrokerService()); - doReturn(FutureUtil.failedFuture(new BrokerServiceException.ServiceUnitNotReadyException("test"))) - .when(brokerService).getManagedLedgerConfig(any()); + AtomicBoolean isGetManagedLedgerConfigFail = new AtomicBoolean(false); + doAnswer(invocation -> { + if (isGetManagedLedgerConfigFail.get()) { + return FutureUtil.failedFuture(new BrokerServiceException.ServiceUnitNotReadyException("test")); + } else { + return invocation.callRealMethod(); + } + }).when(brokerService).getManagedLedgerConfig(any()); Field field = AbstractTopic.class.getDeclaredField("brokerService"); field.setAccessible(true); field.set(persistentTopic, brokerService); @@ -968,11 +975,13 @@ public void testGetManagedLegerConfigFailThenUnload() throws Exception { producer.send("test"); Transaction transaction = pulsarClient.newTransaction() - .withTransactionTimeout(30, TimeUnit.SECONDS).build().get(); + .withTransactionTimeout(10, TimeUnit.SECONDS).build().get(); + isGetManagedLedgerConfigFail.set(true); // pending ack init fail, so the ack will throw exception try { consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); + fail("ack should fail"); } catch (Exception e) { assertTrue(e.getCause() instanceof PulsarClientException.LookupException); } 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 9396a80cf2557..6f79c573ed3d0 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 @@ -291,7 +291,7 @@ public void testTopicLoadAndDeleteAtTheSameTime(boolean injectTimeout) throws Ex Thread.sleep(10 * 1000); } log.info("Race condition occurs {} times", mockRaceConditionCounter.get()); - pulsar.getManagedLedgerFactory().delete(TopicName.get(tpName).getPersistenceNamingEncoding()); + pulsar.getDefaultManagedLedgerFactory().delete(TopicName.get(tpName).getPersistenceNamingEncoding()); } return invocation.callRealMethod(); }).when(namespaceService).isServiceUnitActiveAsync(any(TopicName.class)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 2e71e8cc28c3e..e76c3d8fb845d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -91,8 +91,13 @@ import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.ServerCnx; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.storage.ManagedLedgerStorage; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.broker.testcontext.SpyConfig; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.impl.BatchMessageIdImpl; @@ -150,6 +155,14 @@ protected void setup() throws Exception { super.producerBaseSetup(); } + @Override + protected PulsarTestContext.Builder createPulsarTestContextBuilder(ServiceConfiguration conf) { + return super.createPulsarTestContextBuilder(conf) + .spyConfig(SpyConfig.builder() + .managedLedgerStorage(SpyConfig.SpyType.SPY_ALSO_INVOCATIONS) + .build()); + } + @AfterMethod(alwaysRun = true) public void cleanupAfterMethod() throws Exception { try { @@ -1097,18 +1110,25 @@ public void testSendBigMessageSizeButCompressed() throws Exception { } + @Override protected void beforePulsarStart(PulsarService pulsar) throws Exception { super.beforePulsarStart(pulsar); - doAnswer(i0 -> { - ManagedLedgerFactory factory = (ManagedLedgerFactory) spy(i0.callRealMethod()); - doAnswer(i1 -> { - EntryCacheManager manager = (EntryCacheManager) spy(i1.callRealMethod()); - doAnswer(i2 -> spy(i2.callRealMethod())).when(manager).getEntryCache(any()); - return manager; - }).when(factory).getEntryCacheManager(); - return factory; - }).when(pulsar).getManagedLedgerFactory(); + ManagedLedgerStorage managedLedgerStorage = pulsar.getManagedLedgerStorage(); + doAnswer(invocation -> { + ManagedLedgerStorageClass managedLedgerStorageClass = + (ManagedLedgerStorageClass) spy(invocation.callRealMethod()); + doAnswer(i0 -> { + ManagedLedgerFactory factory = (ManagedLedgerFactory) spy(i0.callRealMethod()); + doAnswer(i1 -> { + EntryCacheManager manager = (EntryCacheManager) spy(i1.callRealMethod()); + doAnswer(i2 -> spy(i2.callRealMethod())).when(manager).getEntryCache(any()); + return manager; + }).when(factory).getEntryCacheManager(); + return factory; + }).when(managedLedgerStorageClass).getManagedLedgerFactory(); + return managedLedgerStorageClass; + }).when(managedLedgerStorage).getDefaultStorageClass(); } /** @@ -1126,6 +1146,7 @@ protected void beforePulsarStart(PulsarService pulsar) throws Exception { public void testActiveAndInActiveConsumerEntryCacheBehavior() throws Exception { log.info("-- Starting {} test --", methodName); + final long batchMessageDelayMs = 100; final int receiverSize = 10; final String topicName = "cache-topic"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java index 36c36735c067e..390e81ad664f9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java @@ -563,7 +563,7 @@ public void testNeverCallCursorIsCursorDataFullyPersistableIfDisabledTheFeature( final String subscription = "s1"; final int msgSendCount = 100; // Inject a injection to record the counter of calling "cursor.isCursorDataFullyPersistable". - final ManagedLedgerImpl ml = (ManagedLedgerImpl) pulsar.getBrokerService().getManagedLedgerFactory().open(mlName); + final ManagedLedgerImpl ml = (ManagedLedgerImpl) pulsar.getDefaultManagedLedgerFactory().open(mlName); final ManagedCursorImpl cursor = (ManagedCursorImpl) ml.openCursor(subscription); final ManagedCursorImpl spyCursor = Mockito.spy(cursor); AtomicInteger callingIsCursorDataFullyPersistableCounter = new AtomicInteger(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerTest.java index d3cb1d60d37ed..0b3ff345acfc3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerTest.java @@ -624,7 +624,7 @@ protected void beforePulsarStart(PulsarService pulsar) throws Exception { return manager; }).when(factory).getEntryCacheManager(); return factory; - }).when(pulsar).getManagedLedgerFactory(); + }).when(pulsar).getDefaultManagedLedgerFactory(); } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java index 1395424b14123..2b1b409b71ce8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java @@ -62,7 +62,7 @@ public void testCheckSequenceId() throws Exception { ManagedLedgerClientFactory clientFactory = new ManagedLedgerClientFactory(); clientFactory.initialize(pulsar.getConfiguration(), pulsar.getLocalMetadataStore(), pulsar.getBookKeeperClientFactory(), eventLoopGroup, OpenTelemetry.noop()); - ManagedLedgerFactory mlFactory = clientFactory.getManagedLedgerFactory(); + ManagedLedgerFactory mlFactory = clientFactory.getDefaultStorageClass().getManagedLedgerFactory(); ManagedLedger ml = mlFactory.open(TopicName.get(topicName).getPersistenceNamingEncoding()); ml.close(); clientFactory.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java index 19f42a7e0570f..d75ccce7ff39c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java @@ -927,7 +927,7 @@ public void testCompactorReadsCompacted() throws Exception { // verify second ledger created String managedLedgerName = ((PersistentTopic)pulsar.getBrokerService().getTopicReference(topic).get()) .getManagedLedger().getName(); - ManagedLedgerInfo info = pulsar.getManagedLedgerFactory().getManagedLedgerInfo(managedLedgerName); + ManagedLedgerInfo info = pulsar.getDefaultManagedLedgerFactory().getManagedLedgerInfo(managedLedgerName); Assert.assertEquals(info.ledgers.size(), 2); Assert.assertTrue(ledgersOpened.isEmpty()); // no ledgers should have been opened @@ -950,7 +950,7 @@ public void testCompactorReadsCompacted() throws Exception { .send(); } - info = pulsar.getManagedLedgerFactory().getManagedLedgerInfo(managedLedgerName); + info = pulsar.getDefaultManagedLedgerFactory().getManagedLedgerInfo(managedLedgerName); Assert.assertEquals(info.ledgers.size(), 3); // should only have opened the penultimate ledger to get stat diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PersistencePolicies.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PersistencePolicies.java index df4e086748f30..3fbc91e7d2eaa 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PersistencePolicies.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PersistencePolicies.java @@ -30,17 +30,24 @@ public class PersistencePolicies { private int bookkeeperWriteQuorum; private int bookkeeperAckQuorum; private double managedLedgerMaxMarkDeleteRate; + private String managedLedgerStorageClassName; public PersistencePolicies() { - this(2, 2, 2, 0.0); + this(2, 2, 2, 0.0, null); } public PersistencePolicies(int bookkeeperEnsemble, int bookkeeperWriteQuorum, int bookkeeperAckQuorum, - double managedLedgerMaxMarkDeleteRate) { + double managedLedgerMaxMarkDeleteRate) { + this(bookkeeperEnsemble, bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, null); + } + + public PersistencePolicies(int bookkeeperEnsemble, int bookkeeperWriteQuorum, int bookkeeperAckQuorum, + double managedLedgerMaxMarkDeleteRate, String managedLedgerStorageClassName) { this.bookkeeperEnsemble = bookkeeperEnsemble; this.bookkeeperWriteQuorum = bookkeeperWriteQuorum; this.bookkeeperAckQuorum = bookkeeperAckQuorum; this.managedLedgerMaxMarkDeleteRate = managedLedgerMaxMarkDeleteRate; + this.managedLedgerStorageClassName = managedLedgerStorageClassName; } public int getBookkeeperEnsemble() { @@ -59,10 +66,14 @@ public double getManagedLedgerMaxMarkDeleteRate() { return managedLedgerMaxMarkDeleteRate; } + public String getManagedLedgerStorageClassName() { + return managedLedgerStorageClassName; + } + @Override public int hashCode() { return Objects.hash(bookkeeperEnsemble, bookkeeperWriteQuorum, - bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate); + bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, managedLedgerStorageClassName); } @Override public boolean equals(Object obj) { @@ -71,7 +82,8 @@ public boolean equals(Object obj) { return bookkeeperEnsemble == other.bookkeeperEnsemble && bookkeeperWriteQuorum == other.bookkeeperWriteQuorum && bookkeeperAckQuorum == other.bookkeeperAckQuorum - && managedLedgerMaxMarkDeleteRate == other.managedLedgerMaxMarkDeleteRate; + && managedLedgerMaxMarkDeleteRate == other.managedLedgerMaxMarkDeleteRate + && managedLedgerStorageClassName == other.managedLedgerStorageClassName; } return false; diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java index e8e644b688029..8adedcd14ac40 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java @@ -1379,6 +1379,11 @@ private class SetPersistence extends CliCommand { description = "Throttling rate of mark-delete operation (0 means no throttle)") private double managedLedgerMaxMarkDeleteRate = 0; + @Option(names = { "-c", + "--ml-storage-class" }, + description = "Managed ledger storage class name") + private String managedLedgerStorageClassName; + @Override void run() throws PulsarAdminException { String namespace = validateNamespace(namespaceName); @@ -1390,7 +1395,8 @@ void run() throws PulsarAdminException { throw new ParameterException("[--ml-mark-delete-max-rate] cannot less than 0."); } getAdmin().namespaces().setPersistence(namespace, new PersistencePolicies(bookkeeperEnsemble, - bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate)); + bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, + managedLedgerStorageClassName)); } } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java index 3cc72db2e95f1..10850d107edf5 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java @@ -1197,6 +1197,11 @@ private class SetPersistence extends CliCommand { + "If set to true, the policy will be replicate to other clusters asynchronously", arity = "0") private boolean isGlobal = false; + @Option(names = { "-c", + "--ml-storage-class" }, + description = "Managed ledger storage class name") + private String managedLedgerStorageClassName; + @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(topicName); @@ -1208,7 +1213,8 @@ void run() throws PulsarAdminException { throw new ParameterException("[--ml-mark-delete-max-rate] cannot less than 0."); } getTopicPolicies(isGlobal).setPersistence(persistentTopic, new PersistencePolicies(bookkeeperEnsemble, - bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate)); + bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, + managedLedgerStorageClassName)); } } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 261bd81a5b7bd..955d6e13e1d04 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -2148,6 +2148,11 @@ private class SetPersistence extends CliCommand { + "(0 means no throttle)") private double managedLedgerMaxMarkDeleteRate = 0; + @Option(names = { "-c", + "--ml-storage-class" }, + description = "Managed ledger storage class name") + private String managedLedgerStorageClassName; + @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(topicName); @@ -2159,7 +2164,8 @@ void run() throws PulsarAdminException { throw new ParameterException("[--ml-mark-delete-max-rate] cannot less than 0."); } getTopics().setPersistence(persistentTopic, new PersistencePolicies(bookkeeperEnsemble, - bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate)); + bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, + managedLedgerStorageClassName)); } } From 06fc259ce568ba6f1dfdee82b2981952357f6e72 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 8 Oct 2024 13:28:16 +0300 Subject: [PATCH 047/327] [improve][build] Update maven-wrapper (mvnw) to recent stable version 3.3.2 (#23410) --- .mvn/wrapper/maven-wrapper.properties | 9 +- mvnw | 457 +++++++++++--------------- mvnw.cmd | 287 +++++++--------- 3 files changed, 329 insertions(+), 424 deletions(-) diff --git a/.mvn/wrapper/maven-wrapper.properties b/.mvn/wrapper/maven-wrapper.properties index db95c131dde6f..d58dfb70bab56 100644 --- a/.mvn/wrapper/maven-wrapper.properties +++ b/.mvn/wrapper/maven-wrapper.properties @@ -5,14 +5,15 @@ # 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. -distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.8.5/apache-maven-3.8.5-bin.zip -wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar +wrapperVersion=3.3.2 +distributionType=only-script +distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.9/apache-maven-3.9.9-bin.zip diff --git a/mvnw b/mvnw index 5643201c7d822..19529ddf8c6ea 100755 --- a/mvnw +++ b/mvnw @@ -19,298 +19,241 @@ # ---------------------------------------------------------------------------- # ---------------------------------------------------------------------------- -# Maven Start Up Batch script -# -# Required ENV vars: -# ------------------ -# JAVA_HOME - location of a JDK home dir +# Apache Maven Wrapper startup batch script, version 3.3.2 # # Optional ENV vars # ----------------- -# M2_HOME - location of maven2's installed home dir -# MAVEN_OPTS - parameters passed to the Java VM when running Maven -# e.g. to debug Maven itself, use -# set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000 -# MAVEN_SKIP_RC - flag to disable loading of mavenrc files +# JAVA_HOME - location of a JDK home dir, required when download maven via java source +# MVNW_REPOURL - repo url base for downloading maven distribution +# MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +# MVNW_VERBOSE - true: enable verbose log; debug: trace the mvnw script; others: silence the output # ---------------------------------------------------------------------------- -if [ -z "$MAVEN_SKIP_RC" ] ; then - - if [ -f /usr/local/etc/mavenrc ] ; then - . /usr/local/etc/mavenrc - fi - - if [ -f /etc/mavenrc ] ; then - . /etc/mavenrc - fi +set -euf +[ "${MVNW_VERBOSE-}" != debug ] || set -x - if [ -f "$HOME/.mavenrc" ] ; then - . "$HOME/.mavenrc" - fi +# OS specific support. +native_path() { printf %s\\n "$1"; } +case "$(uname)" in +CYGWIN* | MINGW*) + [ -z "${JAVA_HOME-}" ] || JAVA_HOME="$(cygpath --unix "$JAVA_HOME")" + native_path() { cygpath --path --windows "$1"; } + ;; +esac -fi +# set JAVACMD and JAVACCMD +set_java_home() { + # For Cygwin and MinGW, ensure paths are in Unix format before anything is touched + if [ -n "${JAVA_HOME-}" ]; then + if [ -x "$JAVA_HOME/jre/sh/java" ]; then + # IBM's JDK on AIX uses strange locations for the executables + JAVACMD="$JAVA_HOME/jre/sh/java" + JAVACCMD="$JAVA_HOME/jre/sh/javac" + else + JAVACMD="$JAVA_HOME/bin/java" + JAVACCMD="$JAVA_HOME/bin/javac" -# OS specific support. $var _must_ be set to either true or false. -cygwin=false; -darwin=false; -mingw=false -case "`uname`" in - CYGWIN*) cygwin=true ;; - MINGW*) mingw=true;; - Darwin*) darwin=true - # Use /usr/libexec/java_home if available, otherwise fall back to /Library/Java/Home - # See https://developer.apple.com/library/mac/qa/qa1170/_index.html - if [ -z "$JAVA_HOME" ]; then - if [ -x "/usr/libexec/java_home" ]; then - export JAVA_HOME="`/usr/libexec/java_home`" - else - export JAVA_HOME="/Library/Java/Home" + if [ ! -x "$JAVACMD" ] || [ ! -x "$JAVACCMD" ]; then + echo "The JAVA_HOME environment variable is not defined correctly, so mvnw cannot run." >&2 + echo "JAVA_HOME is set to \"$JAVA_HOME\", but \"\$JAVA_HOME/bin/java\" or \"\$JAVA_HOME/bin/javac\" does not exist." >&2 + return 1 fi fi - ;; -esac - -if [ -z "$JAVA_HOME" ] ; then - if [ -r /etc/gentoo-release ] ; then - JAVA_HOME=`java-config --jre-home` + else + JAVACMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v java + )" || : + JAVACCMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v javac + )" || : + + if [ ! -x "${JAVACMD-}" ] || [ ! -x "${JAVACCMD-}" ]; then + echo "The java/javac command does not exist in PATH nor is JAVA_HOME set, so mvnw cannot run." >&2 + return 1 + fi fi -fi - -if [ -z "$M2_HOME" ] ; then - ## resolve links - $0 may be a link to maven's home - PRG="$0" +} - # need this for relative symlinks - while [ -h "$PRG" ] ; do - ls=`ls -ld "$PRG"` - link=`expr "$ls" : '.*-> \(.*\)$'` - if expr "$link" : '/.*' > /dev/null; then - PRG="$link" - else - PRG="`dirname "$PRG"`/$link" - fi +# hash string like Java String::hashCode +hash_string() { + str="${1:-}" h=0 + while [ -n "$str" ]; do + char="${str%"${str#?}"}" + h=$(((h * 31 + $(LC_CTYPE=C printf %d "'$char")) % 4294967296)) + str="${str#?}" done + printf %x\\n $h +} - saveddir=`pwd` +verbose() { :; } +[ "${MVNW_VERBOSE-}" != true ] || verbose() { printf %s\\n "${1-}"; } - M2_HOME=`dirname "$PRG"`/.. +die() { + printf %s\\n "$1" >&2 + exit 1 +} - # make it fully qualified - M2_HOME=`cd "$M2_HOME" && pwd` +trim() { + # MWRAPPER-139: + # Trims trailing and leading whitespace, carriage returns, tabs, and linefeeds. + # Needed for removing poorly interpreted newline sequences when running in more + # exotic environments such as mingw bash on Windows. + printf "%s" "${1}" | tr -d '[:space:]' +} - cd "$saveddir" - # echo Using m2 at $M2_HOME -fi +# parse distributionUrl and optional distributionSha256Sum, requires .mvn/wrapper/maven-wrapper.properties +while IFS="=" read -r key value; do + case "${key-}" in + distributionUrl) distributionUrl=$(trim "${value-}") ;; + distributionSha256Sum) distributionSha256Sum=$(trim "${value-}") ;; + esac +done <"${0%/*}/.mvn/wrapper/maven-wrapper.properties" +[ -n "${distributionUrl-}" ] || die "cannot read distributionUrl property in ${0%/*}/.mvn/wrapper/maven-wrapper.properties" + +case "${distributionUrl##*/}" in +maven-mvnd-*bin.*) + MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ + case "${PROCESSOR_ARCHITECTURE-}${PROCESSOR_ARCHITEW6432-}:$(uname -a)" in + *AMD64:CYGWIN* | *AMD64:MINGW*) distributionPlatform=windows-amd64 ;; + :Darwin*x86_64) distributionPlatform=darwin-amd64 ;; + :Darwin*arm64) distributionPlatform=darwin-aarch64 ;; + :Linux*x86_64*) distributionPlatform=linux-amd64 ;; + *) + echo "Cannot detect native platform for mvnd on $(uname)-$(uname -m), use pure java version" >&2 + distributionPlatform=linux-amd64 + ;; + esac + distributionUrl="${distributionUrl%-bin.*}-$distributionPlatform.zip" + ;; +maven-mvnd-*) MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ ;; +*) MVN_CMD="mvn${0##*/mvnw}" _MVNW_REPO_PATTERN=/org/apache/maven/ ;; +esac -# For Cygwin, ensure paths are in UNIX format before anything is touched -if $cygwin ; then - [ -n "$M2_HOME" ] && - M2_HOME=`cygpath --unix "$M2_HOME"` - [ -n "$JAVA_HOME" ] && - JAVA_HOME=`cygpath --unix "$JAVA_HOME"` - [ -n "$CLASSPATH" ] && - CLASSPATH=`cygpath --path --unix "$CLASSPATH"` -fi +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +[ -z "${MVNW_REPOURL-}" ] || distributionUrl="$MVNW_REPOURL$_MVNW_REPO_PATTERN${distributionUrl#*"$_MVNW_REPO_PATTERN"}" +distributionUrlName="${distributionUrl##*/}" +distributionUrlNameMain="${distributionUrlName%.*}" +distributionUrlNameMain="${distributionUrlNameMain%-bin}" +MAVEN_USER_HOME="${MAVEN_USER_HOME:-${HOME}/.m2}" +MAVEN_HOME="${MAVEN_USER_HOME}/wrapper/dists/${distributionUrlNameMain-}/$(hash_string "$distributionUrl")" + +exec_maven() { + unset MVNW_VERBOSE MVNW_USERNAME MVNW_PASSWORD MVNW_REPOURL || : + exec "$MAVEN_HOME/bin/$MVN_CMD" "$@" || die "cannot exec $MAVEN_HOME/bin/$MVN_CMD" +} -# For Mingw, ensure paths are in UNIX format before anything is touched -if $mingw ; then - [ -n "$M2_HOME" ] && - M2_HOME="`(cd "$M2_HOME"; pwd)`" - [ -n "$JAVA_HOME" ] && - JAVA_HOME="`(cd "$JAVA_HOME"; pwd)`" +if [ -d "$MAVEN_HOME" ]; then + verbose "found existing MAVEN_HOME at $MAVEN_HOME" + exec_maven "$@" fi -if [ -z "$JAVA_HOME" ]; then - javaExecutable="`which javac`" - if [ -n "$javaExecutable" ] && ! [ "`expr \"$javaExecutable\" : '\([^ ]*\)'`" = "no" ]; then - # readlink(1) is not available as standard on Solaris 10. - readLink=`which readlink` - if [ ! `expr "$readLink" : '\([^ ]*\)'` = "no" ]; then - if $darwin ; then - javaHome="`dirname \"$javaExecutable\"`" - javaExecutable="`cd \"$javaHome\" && pwd -P`/javac" - else - javaExecutable="`readlink -f \"$javaExecutable\"`" - fi - javaHome="`dirname \"$javaExecutable\"`" - javaHome=`expr "$javaHome" : '\(.*\)/bin'` - JAVA_HOME="$javaHome" - export JAVA_HOME - fi - fi -fi +case "${distributionUrl-}" in +*?-bin.zip | *?maven-mvnd-?*-?*.zip) ;; +*) die "distributionUrl is not valid, must match *-bin.zip or maven-mvnd-*.zip, but found '${distributionUrl-}'" ;; +esac -if [ -z "$JAVACMD" ] ; then - if [ -n "$JAVA_HOME" ] ; then - if [ -x "$JAVA_HOME/jre/sh/java" ] ; then - # IBM's JDK on AIX uses strange locations for the executables - JAVACMD="$JAVA_HOME/jre/sh/java" - else - JAVACMD="$JAVA_HOME/bin/java" - fi - else - JAVACMD="`\\unset -f command; \\command -v java`" - fi +# prepare tmp dir +if TMP_DOWNLOAD_DIR="$(mktemp -d)" && [ -d "$TMP_DOWNLOAD_DIR" ]; then + clean() { rm -rf -- "$TMP_DOWNLOAD_DIR"; } + trap clean HUP INT TERM EXIT +else + die "cannot create temp dir" fi -if [ ! -x "$JAVACMD" ] ; then - echo "Error: JAVA_HOME is not defined correctly." >&2 - echo " We cannot execute $JAVACMD" >&2 - exit 1 -fi +mkdir -p -- "${MAVEN_HOME%/*}" -if [ -z "$JAVA_HOME" ] ; then - echo "Warning: JAVA_HOME environment variable is not set." +# Download and Install Apache Maven +verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +verbose "Downloading from: $distributionUrl" +verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" + +# select .zip or .tar.gz +if ! command -v unzip >/dev/null; then + distributionUrl="${distributionUrl%.zip}.tar.gz" + distributionUrlName="${distributionUrl##*/}" fi -CLASSWORLDS_LAUNCHER=org.codehaus.plexus.classworlds.launcher.Launcher +# verbose opt +__MVNW_QUIET_WGET=--quiet __MVNW_QUIET_CURL=--silent __MVNW_QUIET_UNZIP=-q __MVNW_QUIET_TAR='' +[ "${MVNW_VERBOSE-}" != true ] || __MVNW_QUIET_WGET='' __MVNW_QUIET_CURL='' __MVNW_QUIET_UNZIP='' __MVNW_QUIET_TAR=v -# traverses directory structure from process work directory to filesystem root -# first directory with .mvn subdirectory is considered project base directory -find_maven_basedir() { +# normalize http auth +case "${MVNW_PASSWORD:+has-password}" in +'') MVNW_USERNAME='' MVNW_PASSWORD='' ;; +has-password) [ -n "${MVNW_USERNAME-}" ] || MVNW_USERNAME='' MVNW_PASSWORD='' ;; +esac - if [ -z "$1" ] - then - echo "Path not specified to find_maven_basedir" - return 1 - fi +if [ -z "${MVNW_USERNAME-}" ] && command -v wget >/dev/null; then + verbose "Found wget ... using wget" + wget ${__MVNW_QUIET_WGET:+"$__MVNW_QUIET_WGET"} "$distributionUrl" -O "$TMP_DOWNLOAD_DIR/$distributionUrlName" || die "wget: Failed to fetch $distributionUrl" +elif [ -z "${MVNW_USERNAME-}" ] && command -v curl >/dev/null; then + verbose "Found curl ... using curl" + curl ${__MVNW_QUIET_CURL:+"$__MVNW_QUIET_CURL"} -f -L -o "$TMP_DOWNLOAD_DIR/$distributionUrlName" "$distributionUrl" || die "curl: Failed to fetch $distributionUrl" +elif set_java_home; then + verbose "Falling back to use Java to download" + javaSource="$TMP_DOWNLOAD_DIR/Downloader.java" + targetZip="$TMP_DOWNLOAD_DIR/$distributionUrlName" + cat >"$javaSource" <<-END + public class Downloader extends java.net.Authenticator + { + protected java.net.PasswordAuthentication getPasswordAuthentication() + { + return new java.net.PasswordAuthentication( System.getenv( "MVNW_USERNAME" ), System.getenv( "MVNW_PASSWORD" ).toCharArray() ); + } + public static void main( String[] args ) throws Exception + { + setDefault( new Downloader() ); + java.nio.file.Files.copy( java.net.URI.create( args[0] ).toURL().openStream(), java.nio.file.Paths.get( args[1] ).toAbsolutePath().normalize() ); + } + } + END + # For Cygwin/MinGW, switch paths to Windows format before running javac and java + verbose " - Compiling Downloader.java ..." + "$(native_path "$JAVACCMD")" "$(native_path "$javaSource")" || die "Failed to compile Downloader.java" + verbose " - Running Downloader.java ..." + "$(native_path "$JAVACMD")" -cp "$(native_path "$TMP_DOWNLOAD_DIR")" Downloader "$distributionUrl" "$(native_path "$targetZip")" +fi - basedir="$1" - wdir="$1" - while [ "$wdir" != '/' ] ; do - if [ -d "$wdir"/.mvn ] ; then - basedir=$wdir - break +# If specified, validate the SHA-256 sum of the Maven distribution zip file +if [ -n "${distributionSha256Sum-}" ]; then + distributionSha256Result=false + if [ "$MVN_CMD" = mvnd.sh ]; then + echo "Checksum validation is not supported for maven-mvnd." >&2 + echo "Please disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 + exit 1 + elif command -v sha256sum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | sha256sum -c >/dev/null 2>&1; then + distributionSha256Result=true fi - # workaround for JBEAP-8937 (on Solaris 10/Sparc) - if [ -d "${wdir}" ]; then - wdir=`cd "$wdir/.."; pwd` + elif command -v shasum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | shasum -a 256 -c >/dev/null 2>&1; then + distributionSha256Result=true fi - # end of workaround - done - echo "${basedir}" -} - -# concatenates all lines of a file -concat_lines() { - if [ -f "$1" ]; then - echo "$(tr -s '\n' ' ' < "$1")" + else + echo "Checksum validation was requested but neither 'sha256sum' or 'shasum' are available." >&2 + echo "Please install either command, or disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 + exit 1 + fi + if [ $distributionSha256Result = false ]; then + echo "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised." >&2 + echo "If you updated your Maven version, you need to update the specified distributionSha256Sum property." >&2 + exit 1 fi -} - -BASE_DIR=`find_maven_basedir "$(pwd)"` -if [ -z "$BASE_DIR" ]; then - exit 1; fi -########################################################################################## -# Extension to allow automatically downloading the maven-wrapper.jar from Maven-central -# This allows using the maven wrapper in projects that prohibit checking in binary data. -########################################################################################## -if [ -r "$BASE_DIR/.mvn/wrapper/maven-wrapper.jar" ]; then - if [ "$MVNW_VERBOSE" = true ]; then - echo "Found .mvn/wrapper/maven-wrapper.jar" - fi +# unzip and move +if command -v unzip >/dev/null; then + unzip ${__MVNW_QUIET_UNZIP:+"$__MVNW_QUIET_UNZIP"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -d "$TMP_DOWNLOAD_DIR" || die "failed to unzip" else - if [ "$MVNW_VERBOSE" = true ]; then - echo "Couldn't find .mvn/wrapper/maven-wrapper.jar, downloading it ..." - fi - if [ -n "$MVNW_REPOURL" ]; then - jarUrl="$MVNW_REPOURL/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" - else - jarUrl="https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" - fi - while IFS="=" read key value; do - case "$key" in (wrapperUrl) jarUrl="$value"; break ;; - esac - done < "$BASE_DIR/.mvn/wrapper/maven-wrapper.properties" - if [ "$MVNW_VERBOSE" = true ]; then - echo "Downloading from: $jarUrl" - fi - wrapperJarPath="$BASE_DIR/.mvn/wrapper/maven-wrapper.jar" - if $cygwin; then - wrapperJarPath=`cygpath --path --windows "$wrapperJarPath"` - fi - - if command -v wget > /dev/null; then - if [ "$MVNW_VERBOSE" = true ]; then - echo "Found wget ... using wget" - fi - if [ -z "$MVNW_USERNAME" ] || [ -z "$MVNW_PASSWORD" ]; then - wget "$jarUrl" -O "$wrapperJarPath" || rm -f "$wrapperJarPath" - else - wget --http-user=$MVNW_USERNAME --http-password=$MVNW_PASSWORD "$jarUrl" -O "$wrapperJarPath" || rm -f "$wrapperJarPath" - fi - elif command -v curl > /dev/null; then - if [ "$MVNW_VERBOSE" = true ]; then - echo "Found curl ... using curl" - fi - if [ -z "$MVNW_USERNAME" ] || [ -z "$MVNW_PASSWORD" ]; then - curl -o "$wrapperJarPath" "$jarUrl" -f - else - curl --user $MVNW_USERNAME:$MVNW_PASSWORD -o "$wrapperJarPath" "$jarUrl" -f - fi - - else - if [ "$MVNW_VERBOSE" = true ]; then - echo "Falling back to using Java to download" - fi - javaClass="$BASE_DIR/.mvn/wrapper/MavenWrapperDownloader.java" - # For Cygwin, switch paths to Windows format before running javac - if $cygwin; then - javaClass=`cygpath --path --windows "$javaClass"` - fi - if [ -e "$javaClass" ]; then - if [ ! -e "$BASE_DIR/.mvn/wrapper/MavenWrapperDownloader.class" ]; then - if [ "$MVNW_VERBOSE" = true ]; then - echo " - Compiling MavenWrapperDownloader.java ..." - fi - # Compiling the Java class - ("$JAVA_HOME/bin/javac" "$javaClass") - fi - if [ -e "$BASE_DIR/.mvn/wrapper/MavenWrapperDownloader.class" ]; then - # Running the downloader - if [ "$MVNW_VERBOSE" = true ]; then - echo " - Running MavenWrapperDownloader.java ..." - fi - ("$JAVA_HOME/bin/java" -cp .mvn/wrapper MavenWrapperDownloader "$MAVEN_PROJECTBASEDIR") - fi - fi - fi -fi -########################################################################################## -# End of extension -########################################################################################## - -export MAVEN_PROJECTBASEDIR=${MAVEN_BASEDIR:-"$BASE_DIR"} -if [ "$MVNW_VERBOSE" = true ]; then - echo $MAVEN_PROJECTBASEDIR + tar xzf${__MVNW_QUIET_TAR:+"$__MVNW_QUIET_TAR"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -C "$TMP_DOWNLOAD_DIR" || die "failed to untar" fi -MAVEN_OPTS="$(concat_lines "$MAVEN_PROJECTBASEDIR/.mvn/jvm.config") $MAVEN_OPTS" - -# For Cygwin, switch paths to Windows format before running java -if $cygwin; then - [ -n "$M2_HOME" ] && - M2_HOME=`cygpath --path --windows "$M2_HOME"` - [ -n "$JAVA_HOME" ] && - JAVA_HOME=`cygpath --path --windows "$JAVA_HOME"` - [ -n "$CLASSPATH" ] && - CLASSPATH=`cygpath --path --windows "$CLASSPATH"` - [ -n "$MAVEN_PROJECTBASEDIR" ] && - MAVEN_PROJECTBASEDIR=`cygpath --path --windows "$MAVEN_PROJECTBASEDIR"` -fi - -# Provide a "standardized" way to retrieve the CLI args that will -# work with both Windows and non-Windows executions. -MAVEN_CMD_LINE_ARGS="$MAVEN_CONFIG $@" -export MAVEN_CMD_LINE_ARGS - -WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain +printf %s\\n "$distributionUrl" >"$TMP_DOWNLOAD_DIR/$distributionUrlNameMain/mvnw.url" +mv -- "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" "$MAVEN_HOME" || [ -d "$MAVEN_HOME" ] || die "fail to move MAVEN_HOME" -exec "$JAVACMD" \ - $MAVEN_OPTS \ - $MAVEN_DEBUG_OPTS \ - -classpath "$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.jar" \ - "-Dmaven.home=${M2_HOME}" \ - "-Dmaven.multiModuleProjectDirectory=${MAVEN_PROJECTBASEDIR}" \ - ${WRAPPER_LAUNCHER} $MAVEN_CONFIG "$@" +clean || : +exec_maven "$@" diff --git a/mvnw.cmd b/mvnw.cmd index 23b7079a3d4c7..b150b91ed5005 100644 --- a/mvnw.cmd +++ b/mvnw.cmd @@ -1,3 +1,4 @@ +<# : batch portion @REM ---------------------------------------------------------------------------- @REM Licensed to the Apache Software Foundation (ASF) under one @REM or more contributor license agreements. See the NOTICE file @@ -18,171 +19,131 @@ @REM ---------------------------------------------------------------------------- @REM ---------------------------------------------------------------------------- -@REM Maven Start Up Batch script -@REM -@REM Required ENV vars: -@REM JAVA_HOME - location of a JDK home dir +@REM Apache Maven Wrapper startup batch script, version 3.3.2 @REM @REM Optional ENV vars -@REM M2_HOME - location of maven2's installed home dir -@REM MAVEN_BATCH_ECHO - set to 'on' to enable the echoing of the batch commands -@REM MAVEN_BATCH_PAUSE - set to 'on' to wait for a keystroke before ending -@REM MAVEN_OPTS - parameters passed to the Java VM when running Maven -@REM e.g. to debug Maven itself, use -@REM set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000 -@REM MAVEN_SKIP_RC - flag to disable loading of mavenrc files +@REM MVNW_REPOURL - repo url base for downloading maven distribution +@REM MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +@REM MVNW_VERBOSE - true: enable verbose log; others: silence the output @REM ---------------------------------------------------------------------------- -@REM Begin all REM lines with '@' in case MAVEN_BATCH_ECHO is 'on' -@echo off -@REM set title of command window -title %0 -@REM enable echoing by setting MAVEN_BATCH_ECHO to 'on' -@if "%MAVEN_BATCH_ECHO%" == "on" echo %MAVEN_BATCH_ECHO% - -@REM set %HOME% to equivalent of $HOME -if "%HOME%" == "" (set "HOME=%HOMEDRIVE%%HOMEPATH%") - -@REM Execute a user defined script before this one -if not "%MAVEN_SKIP_RC%" == "" goto skipRcPre -@REM check for pre script, once with legacy .bat ending and once with .cmd ending -if exist "%USERPROFILE%\mavenrc_pre.bat" call "%USERPROFILE%\mavenrc_pre.bat" %* -if exist "%USERPROFILE%\mavenrc_pre.cmd" call "%USERPROFILE%\mavenrc_pre.cmd" %* -:skipRcPre - -@setlocal - -set ERROR_CODE=0 - -@REM To isolate internal variables from possible post scripts, we use another setlocal -@setlocal - -@REM ==== START VALIDATION ==== -if not "%JAVA_HOME%" == "" goto OkJHome - -echo. -echo Error: JAVA_HOME not found in your environment. >&2 -echo Please set the JAVA_HOME variable in your environment to match the >&2 -echo location of your Java installation. >&2 -echo. -goto error - -:OkJHome -if exist "%JAVA_HOME%\bin\java.exe" goto init - -echo. -echo Error: JAVA_HOME is set to an invalid directory. >&2 -echo JAVA_HOME = "%JAVA_HOME%" >&2 -echo Please set the JAVA_HOME variable in your environment to match the >&2 -echo location of your Java installation. >&2 -echo. -goto error - -@REM ==== END VALIDATION ==== - -:init - -@REM Find the project base dir, i.e. the directory that contains the folder ".mvn". -@REM Fallback to current working directory if not found. - -set MAVEN_PROJECTBASEDIR=%MAVEN_BASEDIR% -IF NOT "%MAVEN_PROJECTBASEDIR%"=="" goto endDetectBaseDir - -set EXEC_DIR=%CD% -set WDIR=%EXEC_DIR% -:findBaseDir -IF EXIST "%WDIR%"\.mvn goto baseDirFound -cd .. -IF "%WDIR%"=="%CD%" goto baseDirNotFound -set WDIR=%CD% -goto findBaseDir - -:baseDirFound -set MAVEN_PROJECTBASEDIR=%WDIR% -cd "%EXEC_DIR%" -goto endDetectBaseDir - -:baseDirNotFound -set MAVEN_PROJECTBASEDIR=%EXEC_DIR% -cd "%EXEC_DIR%" - -:endDetectBaseDir - -IF NOT EXIST "%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config" goto endReadAdditionalConfig - -@setlocal EnableExtensions EnableDelayedExpansion -for /F "usebackq delims=" %%a in ("%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config") do set JVM_CONFIG_MAVEN_PROPS=!JVM_CONFIG_MAVEN_PROPS! %%a -@endlocal & set JVM_CONFIG_MAVEN_PROPS=%JVM_CONFIG_MAVEN_PROPS% - -:endReadAdditionalConfig - -SET MAVEN_JAVA_EXE="%JAVA_HOME%\bin\java.exe" -set WRAPPER_JAR="%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.jar" -set WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain - -set DOWNLOAD_URL="https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" - -FOR /F "usebackq tokens=1,2 delims==" %%A IN ("%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.properties") DO ( - IF "%%A"=="wrapperUrl" SET DOWNLOAD_URL=%%B -) - -@REM Extension to allow automatically downloading the maven-wrapper.jar from Maven-central -@REM This allows using the maven wrapper in projects that prohibit checking in binary data. -if exist %WRAPPER_JAR% ( - if "%MVNW_VERBOSE%" == "true" ( - echo Found %WRAPPER_JAR% - ) -) else ( - if not "%MVNW_REPOURL%" == "" ( - SET DOWNLOAD_URL="%MVNW_REPOURL%/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" - ) - if "%MVNW_VERBOSE%" == "true" ( - echo Couldn't find %WRAPPER_JAR%, downloading it ... - echo Downloading from: %DOWNLOAD_URL% - ) - - powershell -Command "&{"^ - "$webclient = new-object System.Net.WebClient;"^ - "if (-not ([string]::IsNullOrEmpty('%MVNW_USERNAME%') -and [string]::IsNullOrEmpty('%MVNW_PASSWORD%'))) {"^ - "$webclient.Credentials = new-object System.Net.NetworkCredential('%MVNW_USERNAME%', '%MVNW_PASSWORD%');"^ - "}"^ - "[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12; $webclient.DownloadFile('%DOWNLOAD_URL%', '%WRAPPER_JAR%')"^ - "}" - if "%MVNW_VERBOSE%" == "true" ( - echo Finished downloading %WRAPPER_JAR% - ) +@IF "%__MVNW_ARG0_NAME__%"=="" (SET __MVNW_ARG0_NAME__=%~nx0) +@SET __MVNW_CMD__= +@SET __MVNW_ERROR__= +@SET __MVNW_PSMODULEP_SAVE=%PSModulePath% +@SET PSModulePath= +@FOR /F "usebackq tokens=1* delims==" %%A IN (`powershell -noprofile "& {$scriptDir='%~dp0'; $script='%__MVNW_ARG0_NAME__%'; icm -ScriptBlock ([Scriptblock]::Create((Get-Content -Raw '%~f0'))) -NoNewScope}"`) DO @( + IF "%%A"=="MVN_CMD" (set __MVNW_CMD__=%%B) ELSE IF "%%B"=="" (echo %%A) ELSE (echo %%A=%%B) ) -@REM End of extension - -@REM Provide a "standardized" way to retrieve the CLI args that will -@REM work with both Windows and non-Windows executions. -set MAVEN_CMD_LINE_ARGS=%* - -%MAVEN_JAVA_EXE% ^ - %JVM_CONFIG_MAVEN_PROPS% ^ - %MAVEN_OPTS% ^ - %MAVEN_DEBUG_OPTS% ^ - -classpath %WRAPPER_JAR% ^ - "-Dmaven.multiModuleProjectDirectory=%MAVEN_PROJECTBASEDIR%" ^ - %WRAPPER_LAUNCHER% %MAVEN_CONFIG% %* -if ERRORLEVEL 1 goto error -goto end - -:error -set ERROR_CODE=1 - -:end -@endlocal & set ERROR_CODE=%ERROR_CODE% - -if not "%MAVEN_SKIP_RC%"=="" goto skipRcPost -@REM check for post script, once with legacy .bat ending and once with .cmd ending -if exist "%USERPROFILE%\mavenrc_post.bat" call "%USERPROFILE%\mavenrc_post.bat" -if exist "%USERPROFILE%\mavenrc_post.cmd" call "%USERPROFILE%\mavenrc_post.cmd" -:skipRcPost - -@REM pause the script if MAVEN_BATCH_PAUSE is set to 'on' -if "%MAVEN_BATCH_PAUSE%"=="on" pause - -if "%MAVEN_TERMINATE_CMD%"=="on" exit %ERROR_CODE% - -cmd /C exit /B %ERROR_CODE% +@SET PSModulePath=%__MVNW_PSMODULEP_SAVE% +@SET __MVNW_PSMODULEP_SAVE= +@SET __MVNW_ARG0_NAME__= +@SET MVNW_USERNAME= +@SET MVNW_PASSWORD= +@IF NOT "%__MVNW_CMD__%"=="" (%__MVNW_CMD__% %*) +@echo Cannot start maven from wrapper >&2 && exit /b 1 +@GOTO :EOF +: end batch / begin powershell #> + +$ErrorActionPreference = "Stop" +if ($env:MVNW_VERBOSE -eq "true") { + $VerbosePreference = "Continue" +} + +# calculate distributionUrl, requires .mvn/wrapper/maven-wrapper.properties +$distributionUrl = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionUrl +if (!$distributionUrl) { + Write-Error "cannot read distributionUrl property in $scriptDir/.mvn/wrapper/maven-wrapper.properties" +} + +switch -wildcard -casesensitive ( $($distributionUrl -replace '^.*/','') ) { + "maven-mvnd-*" { + $USE_MVND = $true + $distributionUrl = $distributionUrl -replace '-bin\.[^.]*$',"-windows-amd64.zip" + $MVN_CMD = "mvnd.cmd" + break + } + default { + $USE_MVND = $false + $MVN_CMD = $script -replace '^mvnw','mvn' + break + } +} + +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +if ($env:MVNW_REPOURL) { + $MVNW_REPO_PATTERN = if ($USE_MVND) { "/org/apache/maven/" } else { "/maven/mvnd/" } + $distributionUrl = "$env:MVNW_REPOURL$MVNW_REPO_PATTERN$($distributionUrl -replace '^.*'+$MVNW_REPO_PATTERN,'')" +} +$distributionUrlName = $distributionUrl -replace '^.*/','' +$distributionUrlNameMain = $distributionUrlName -replace '\.[^.]*$','' -replace '-bin$','' +$MAVEN_HOME_PARENT = "$HOME/.m2/wrapper/dists/$distributionUrlNameMain" +if ($env:MAVEN_USER_HOME) { + $MAVEN_HOME_PARENT = "$env:MAVEN_USER_HOME/wrapper/dists/$distributionUrlNameMain" +} +$MAVEN_HOME_NAME = ([System.Security.Cryptography.MD5]::Create().ComputeHash([byte[]][char[]]$distributionUrl) | ForEach-Object {$_.ToString("x2")}) -join '' +$MAVEN_HOME = "$MAVEN_HOME_PARENT/$MAVEN_HOME_NAME" + +if (Test-Path -Path "$MAVEN_HOME" -PathType Container) { + Write-Verbose "found existing MAVEN_HOME at $MAVEN_HOME" + Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" + exit $? +} + +if (! $distributionUrlNameMain -or ($distributionUrlName -eq $distributionUrlNameMain)) { + Write-Error "distributionUrl is not valid, must end with *-bin.zip, but found $distributionUrl" +} + +# prepare tmp dir +$TMP_DOWNLOAD_DIR_HOLDER = New-TemporaryFile +$TMP_DOWNLOAD_DIR = New-Item -Itemtype Directory -Path "$TMP_DOWNLOAD_DIR_HOLDER.dir" +$TMP_DOWNLOAD_DIR_HOLDER.Delete() | Out-Null +trap { + if ($TMP_DOWNLOAD_DIR.Exists) { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } + } +} + +New-Item -Itemtype Directory -Path "$MAVEN_HOME_PARENT" -Force | Out-Null + +# Download and Install Apache Maven +Write-Verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +Write-Verbose "Downloading from: $distributionUrl" +Write-Verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" + +$webclient = New-Object System.Net.WebClient +if ($env:MVNW_USERNAME -and $env:MVNW_PASSWORD) { + $webclient.Credentials = New-Object System.Net.NetworkCredential($env:MVNW_USERNAME, $env:MVNW_PASSWORD) +} +[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12 +$webclient.DownloadFile($distributionUrl, "$TMP_DOWNLOAD_DIR/$distributionUrlName") | Out-Null + +# If specified, validate the SHA-256 sum of the Maven distribution zip file +$distributionSha256Sum = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionSha256Sum +if ($distributionSha256Sum) { + if ($USE_MVND) { + Write-Error "Checksum validation is not supported for maven-mvnd. `nPlease disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." + } + Import-Module $PSHOME\Modules\Microsoft.PowerShell.Utility -Function Get-FileHash + if ((Get-FileHash "$TMP_DOWNLOAD_DIR/$distributionUrlName" -Algorithm SHA256).Hash.ToLower() -ne $distributionSha256Sum) { + Write-Error "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised. If you updated your Maven version, you need to update the specified distributionSha256Sum property." + } +} + +# unzip and move +Expand-Archive "$TMP_DOWNLOAD_DIR/$distributionUrlName" -DestinationPath "$TMP_DOWNLOAD_DIR" | Out-Null +Rename-Item -Path "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" -NewName $MAVEN_HOME_NAME | Out-Null +try { + Move-Item -Path "$TMP_DOWNLOAD_DIR/$MAVEN_HOME_NAME" -Destination $MAVEN_HOME_PARENT | Out-Null +} catch { + if (! (Test-Path -Path "$MAVEN_HOME" -PathType Container)) { + Write-Error "fail to move MAVEN_HOME" + } +} finally { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } +} + +Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" From c598974346a5df2bb328e679f49b6cb0b56ab84b Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Tue, 8 Oct 2024 19:22:31 +0800 Subject: [PATCH 048/327] [improve][PIP] PIP-383: Support granting/revoking permissions for multiple topics (#23355) --- pip/pip-383.md | 144 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 144 insertions(+) create mode 100644 pip/pip-383.md diff --git a/pip/pip-383.md b/pip/pip-383.md new file mode 100644 index 0000000000000..72f4e182ea7ea --- /dev/null +++ b/pip/pip-383.md @@ -0,0 +1,144 @@ +# PIP-383: Support granting/revoking permissions for multiple topics + +## Background + +In AuthorizationProvider, the authorization interface `grantPermissionAsync(TopicName topicName, Set actions, String role, String authDataJson)` currently only supports granting permissions to a single topic at a time. +When multiple topics need to be authorized under a namespace, the client makes the calls to the authorization interface concurrently. +Since the permissions information is stored in the namespace-level policies, and multiple topics may be on different brokers, concurrent authorization modification will cause concurrent modification exceptions. +Therefore, supporting granting permissions for multiple topics is very beneficial. + + +## Motivation + +Supporting granting/revoking permissions for multiple topics, +add `grantPermissionAsync(List options)` and `revokePermissionAsync(List options)` in AuthorizationProvider. + +## Goals + +### In Scope + +- Add `grantPermissionAsync(List options)` in AuthorizationProvider. +- Add `revokePermissionAsync(List options)` in AuthorizationProvider. + +## High-Level Design + +### Design & Implementation Details + +Add default method implementation in AuthorizationProvider +```java + +public interface AuthorizationProvider extends Closeable { + + default CompletableFuture grantPermissionAsync(List options) { + return FutureUtil.failedFuture(new IllegalStateException( + String.format("grantPermissionAsync is not supported by the Authorization"))); + } + + default CompletableFuture revokePermissionAsync(List options) { + return FutureUtil.failedFuture(new IllegalStateException( + String.format("revokePermissionAsync is not supported by the Authorization"))); + } +} +``` + +``` +@Data +@Builder +public class GrantTopicPermissionOptions { + + private final String topic; + + private final String role; + + private final Set actions; +} + +@Data +@Builder +public class RevokeTopicPermissionOptions { + + private final String topic; + + private final String role; +} +``` + +Add namespace admin API. + +```java +public interface Namespaces { + + CompletableFuture grantPermissionOnTopicsAsync(List options); + + void grantPermissionOnTopics(List options) throws PulsarAdminException; + + CompletableFuture revokePermissionOnTopicsAsync(List options); + + void revokePermissionOnTopics(List options) throws PulsarAdminException; +} +``` + +Add namespace rest implementation in broker side. +```java +@POST +@Path("/grantPermissions") +public void grantPermissionOnTopics(@Suspended final AsyncResponse asyncResponse, + List options) { + internalGrantPermissionsAsync(options) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + log.error("[{}] Failed to grant permissions {}", + clientAppId(), options, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); +} + +@POST +@Path("/revokePermissions") +public void revokePermissionOnTopics(@Suspended final AsyncResponse asyncResponse, + List options) { + internalRevokePermissionsAsync(options) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + log.error("[{}] Failed to revoke permissions {}", + clientAppId(), options, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); +} +``` + +so user can grant/revoke permissions to multi-topics like : +```java +public class TestAuthorization { + + @Test + public void testGrantPermission() { + // grant permission for multi-topics + List grantPermissions = new ArrayList<>(); + grantPermissions.add(GrantPermissionOptions.builder().topic("topic1").role("role1").actions(Set.of(AuthAction.produce)).build()); + grantPermissions.add(GrantPermissionOptions.builder().topic("topic2").role("role2").actions(Set.of(AuthAction.consume)).build()); + admin.namespaces().grantPermissionOnTopics(grantPermissions); + // revoke permission topics + List revokePermissions = new ArrayList<>(); + revokePermissions.add(RevokePermissionOptions.builder().topic("topic1").role("role1").build()); + revokePermissions.add(RevokePermissionOptions.builder().topic("topic2").role("role2").build()); + admin.namespaces().revokePermissionOnTopics(revokePermissions); + } +} + +``` + +## Backward & Forward Compatibility + + + +## Alternatives + +## General Notes + +## Links + +* Mailing List discussion thread: https://lists.apache.org/thread/6n2jdl9bsf1f6xz2orygz3kvxmy11ykh +* Mailing List voting thread: https://lists.apache.org/thread/qbyvs75r0d64h6jk8w1swr782l85b77h From 4efcc1502f5d56047094113a7f14468c1ef90a05 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 8 Oct 2024 15:40:38 +0300 Subject: [PATCH 049/327] [improve][ci] Move some flaky ExtensibleLoadManager tests to flaky group until they are fixed (#23414) --- .../loadbalance/extensions/BrokerRegistryIntegrationTest.java | 2 +- .../extensions/BrokerRegistryMetadataStoreIntegrationTest.java | 2 +- .../loadbalance/extensions/ExtensibleLoadManagerCloseTest.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java index e975671fa12e8..189c29400c4f9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -37,7 +37,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "broker") +@Test(groups = "flaky") public class BrokerRegistryIntegrationTest { private static final String clusterName = "test"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java index 3e01b1fad0f21..15097b565db6c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java @@ -22,7 +22,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl; import org.testng.annotations.Test; -@Test(groups = "broker") +@Test(groups = "flaky") public class BrokerRegistryMetadataStoreIntegrationTest extends BrokerRegistryIntegrationTest { @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java index fa63ce566c603..ca44f6bc4d6d9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java @@ -38,7 +38,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "broker") +@Test(groups = "flaky") public class ExtensibleLoadManagerCloseTest { private static final String clusterName = "test"; From 3d0625ba64294fb0fe7dafc27c7a34883b4be51b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 8 Oct 2024 17:03:13 +0300 Subject: [PATCH 050/327] [improve][broker] PIP-379: Key_Shared Draining Hashes for Improved Message Ordering (#23352) --- .../server/src/assemble/LICENSE.bin.txt | 1 + pom.xml | 7 + .../pulsar/broker/ServiceConfiguration.java | 10 + pulsar-broker/pom.xml | 5 + .../AbstractDelayedDeliveryTracker.java | 2 +- .../pulsar/broker/service/BrokerService.java | 2 +- ...stentHashingStickyKeyConsumerSelector.java | 105 ++- .../pulsar/broker/service/Consumer.java | 134 +-- .../ConsumerHashAssignmentsSnapshot.java | 224 +++++ .../broker/service/DrainingHashesTracker.java | 241 ++++++ .../broker/service/EntryAndMetadata.java | 46 +- .../broker/service/HashRangeAssignment.java | 26 + ...ngeAutoSplitStickyKeyConsumerSelector.java | 47 +- ...ngeExclusiveStickyKeyConsumerSelector.java | 48 +- .../service/ImpactedConsumersResult.java | 59 ++ .../pulsar/broker/service/PendingAcksMap.java | 424 ++++++++++ .../pulsar/broker/service/Producer.java | 20 +- .../broker/service/RemovedHashRanges.java | 74 ++ .../service/StickyKeyConsumerSelector.java | 74 +- .../StickyKeyConsumerSelectorUtils.java | 51 ++ ...tStickyKeyDispatcherMultipleConsumers.java | 8 +- .../MessageRedeliveryController.java | 24 +- ...PersistentDispatcherMultipleConsumers.java | 157 ++-- ...tStickyKeyDispatcherMultipleConsumers.java | 480 ++++------- .../persistent/PersistentSubscription.java | 26 - .../persistent/RescheduleReadHandler.java | 102 +++ .../ConcurrentBitmapSortedLongPairSet.java | 14 +- .../apache/pulsar/broker/BrokerTestUtil.java | 19 + .../pulsar/broker/admin/AdminApiTest.java | 206 ----- .../delayed/AbstractDeliveryTrackerTest.java | 14 +- ...tHashingStickyKeyConsumerSelectorTest.java | 85 +- .../ConsumerHashAssignmentsSnapshotTest.java | 204 +++++ .../service/DrainingHashesTrackerTest.java | 213 +++++ .../broker/service/PendingAcksMapTest.java | 196 +++++ .../service/PersistentTopicE2ETest.java | 87 -- ...istentDispatcherMultipleConsumersTest.java | 4 +- ...ckyKeyDispatcherMultipleConsumersTest.java | 469 ++--------- .../persistent/RescheduleReadHandlerTest.java | 161 ++++ .../broker/stats/SubscriptionStatsTest.java | 44 - .../client/api/KeySharedSubscriptionTest.java | 778 +++++++----------- .../impl/KeySharedSubscriptionTest.java | 161 +++- .../policies/data/SubscriptionStats.java | 9 - .../org/apache/pulsar/client/api/Range.java | 8 + .../apache/pulsar/client/api/RangeTest.java | 40 + .../data/stats/SubscriptionStatsImpl.java | 10 - .../pulsar/common/protocol/Commands.java | 38 +- .../src/main/resources/findbugsExclude.xml | 5 + 47 files changed, 3324 insertions(+), 1838 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 24eb6b8066df1..271f6dc6cebf7 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -262,6 +262,7 @@ The Apache Software License, Version 2.0 - com.fasterxml.jackson.module-jackson-module-parameter-names-2.17.2.jar * Caffeine -- com.github.ben-manes.caffeine-caffeine-2.9.1.jar * Conscrypt -- org.conscrypt-conscrypt-openjdk-uber-2.5.2.jar + * Fastutil -- it.unimi.dsi-fastutil-8.5.14.jar * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.17.0.jar * Bitbucket -- org.bitbucket.b_c-jose4j-0.9.4.jar * Gson diff --git a/pom.xml b/pom.xml index e0bce0442e158..f99eb3066d5e6 100644 --- a/pom.xml +++ b/pom.xml @@ -162,6 +162,7 @@ flexible messaging model and an intuitive client API. 1.0.7 1.0.2.5 2.17.2 + 8.5.14 0.10.2 1.6.2 10.14.2 @@ -911,6 +912,12 @@ flexible messaging model and an intuitive client API. import + + it.unimi.dsi + fastutil + ${fastutil.version} + + org.codehaus.jettison jettison diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 81073b1731b24..1b021bd569969 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -979,6 +979,16 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece ) private int keySharedLookAheadMsgInReplayThresholdPerSubscription = 20000; + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For Key_Shared subscriptions, when a blocked key hash gets unblocked," + + " a redelivery will be attempted after a delay. This setting controls the delay." + + " The reason to have the delay is to batch multiple unblocking events instead of triggering" + + " redelivery for each unblocking event.", + dynamic = true + ) + private long keySharedUnblockingIntervalMs = 10L; + @FieldContext( category = CATEGORY_POLICIES, doc = "Once broker reaches maxUnackedMessagesPerBroker limit, it blocks subscriptions which has higher " diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index ee22762719175..a9521e76296de 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -63,6 +63,11 @@ protobuf-java + + it.unimi.dsi + fastutil + + ${project.groupId} pulsar-client-original diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java index 5c99e4c307d7c..f93a627bca7b8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java @@ -146,7 +146,7 @@ public void run(Timeout timeout) throws Exception { lastTickRun = clock.millis(); currentTimeoutTarget = -1; this.timeout = null; - dispatcher.readMoreEntries(); + dispatcher.readMoreEntriesAsync(); } } 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 ed0cdf18b47ca..8d0b9a4a84e6a 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 @@ -3416,7 +3416,7 @@ public void unblockDispatchersOnUnAckMessages(List { dispatcher.unBlockDispatcherOnUnackedMsgs(); - executor().execute(() -> dispatcher.readMoreEntries()); + dispatcher.readMoreEntriesAsync(); log.info("[{}] Dispatcher is unblocked", dispatcher.getName()); blockedDispatchers.remove(dispatcher); }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java index 1ae9a6ff96b7d..8381f9543bdc2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.service; import java.util.ArrayList; -import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -28,13 +27,10 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.pulsar.client.api.Range; -import org.apache.pulsar.common.util.Murmur3_32Hash; /** - * This is a consumer selector based fixed hash range. - * - * The implementation uses consistent hashing to evenly split, the - * number of keys assigned to each consumer. + * This is a consumer selector using consistent hashing to evenly split + * the number of keys assigned to each consumer. */ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyConsumerSelector { // use NUL character as field separator for hash key calculation @@ -47,14 +43,22 @@ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyCons private final ConsumerNameIndexTracker consumerNameIndexTracker = new ConsumerNameIndexTracker(); private final int numberOfPoints; + private final Range keyHashRange; + private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints) { + this(numberOfPoints, DEFAULT_RANGE_SIZE - 1); + } + + public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints, int rangeMaxValue) { this.hashRing = new TreeMap<>(); this.numberOfPoints = numberOfPoints; + this.keyHashRange = Range.of(STICKY_KEY_HASH_NOT_SET + 1, rangeMaxValue); + this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); } @Override - public CompletableFuture addConsumer(Consumer consumer) { + public CompletableFuture addConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); @@ -72,7 +76,11 @@ public CompletableFuture addConsumer(Consumer consumer) { consumerNameIndexTracker.decreaseConsumerRefCount(removed); } } - return CompletableFuture.completedFuture(null); + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return CompletableFuture.completedFuture(impactedConsumers); } finally { rwLock.writeLock().unlock(); } @@ -88,14 +96,14 @@ public CompletableFuture addConsumer(Consumer consumer) { * @param hashRingPointIndex the index of the hash ring point * @return the hash value */ - private static int calculateHashForConsumerAndIndex(Consumer consumer, int consumerNameIndex, + private int calculateHashForConsumerAndIndex(Consumer consumer, int consumerNameIndex, int hashRingPointIndex) { String key = consumer.consumerName() + KEY_SEPARATOR + consumerNameIndex + KEY_SEPARATOR + hashRingPointIndex; - return Murmur3_32Hash.getInstance().makeHash(key.getBytes()); + return makeStickyKeyHash(key.getBytes()); } @Override - public void removeConsumer(Consumer consumer) { + public ImpactedConsumersResult removeConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); @@ -109,6 +117,11 @@ public void removeConsumer(Consumer consumer) { } } } + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return impactedConsumers; } finally { rwLock.writeLock().unlock(); } @@ -134,32 +147,58 @@ public Consumer select(int hash) { } @Override - public Map> getConsumerKeyHashRanges() { - Map> result = new IdentityHashMap<>(); + public Range getKeyHashRange() { + return keyHashRange; + } + + @Override + public ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { rwLock.readLock().lock(); try { - if (hashRing.isEmpty()) { - return result; - } - int start = 0; - int lastKey = 0; - for (Map.Entry entry: hashRing.entrySet()) { - Consumer consumer = entry.getValue().consumer; - result.computeIfAbsent(consumer, key -> new ArrayList<>()) - .add(Range.of(start, entry.getKey())); - lastKey = entry.getKey(); - start = lastKey + 1; - } - // Handle wrap-around in the hash ring, the first consumer will also contain the range from the last key - // to the maximum value of the hash range - Consumer firstConsumer = hashRing.firstEntry().getValue().consumer; - List ranges = result.get(firstConsumer); - if (lastKey != Integer.MAX_VALUE - 1) { - ranges.add(Range.of(lastKey + 1, Integer.MAX_VALUE - 1)); - } + return consumerHashAssignmentsSnapshot; } finally { rwLock.readLock().unlock(); } - return result; + } + + private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { + if (hashRing.isEmpty()) { + return ConsumerHashAssignmentsSnapshot.empty(); + } + List result = new ArrayList<>(); + int start = getKeyHashRange().getStart(); + int lastKey = -1; + Consumer previousConsumer = null; + Range previousRange = null; + for (Map.Entry entry: hashRing.entrySet()) { + Consumer consumer = entry.getValue().consumer; + Range range; + if (consumer == previousConsumer) { + // join ranges + result.remove(result.size() - 1); + range = Range.of(previousRange.getStart(), entry.getKey()); + } else { + range = Range.of(start, entry.getKey()); + } + result.add(new HashRangeAssignment(range, consumer)); + lastKey = entry.getKey(); + start = lastKey + 1; + previousConsumer = consumer; + previousRange = range; + } + // Handle wrap-around + Consumer firstConsumer = hashRing.firstEntry().getValue().consumer; + if (lastKey != getKeyHashRange().getEnd()) { + Range range; + if (firstConsumer == previousConsumer && previousRange.getEnd() == lastKey) { + // join ranges + result.remove(result.size() - 1); + range = Range.of(previousRange.getStart(), getKeyHashRange().getEnd()); + } else { + range = Range.of(lastKey + 1, getKeyHashRange().getEnd()); + } + result.add(new HashRangeAssignment(range, firstConsumer)); + } + return ConsumerHashAssignmentsSnapshot.of(result); } } 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 7f46e8969eb53..c9584f2c1790f 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 @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; @@ -26,6 +27,8 @@ import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; import io.opentelemetry.api.common.Attributes; +import it.unimi.dsi.fastutil.ints.IntIntPair; +import it.unimi.dsi.fastutil.objects.ObjectIntPair; import java.time.Instant; import java.util.ArrayList; import java.util.BitSet; @@ -65,14 +68,11 @@ import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; -import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.stats.Rate; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair; import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.transaction.common.exception.TransactionConflictException; import org.slf4j.Logger; @@ -119,7 +119,7 @@ public class Consumer { AtomicIntegerFieldUpdater.newUpdater(Consumer.class, "permitsReceivedWhileConsumerBlocked"); private volatile int permitsReceivedWhileConsumerBlocked = 0; - private final ConcurrentLongLongPairHashMap pendingAcks; + private final PendingAcksMap pendingAcks; private final ConsumerStatsImpl stats; @@ -167,6 +167,13 @@ public class Consumer { private static final AtomicReferenceFieldUpdater OPEN_TELEMETRY_ATTRIBUTES_FIELD_UPDATER = AtomicReferenceFieldUpdater.newUpdater(Consumer.class, Attributes.class, "openTelemetryAttributes"); + @Getter + @Setter + private volatile PendingAcksMap.PendingAcksAddHandler pendingAcksAddHandler; + @Getter + @Setter + private volatile PendingAcksMap.PendingAcksRemoveHandler pendingAcksRemoveHandler; + public Consumer(Subscription subscription, SubType subType, String topicName, long consumerId, int priorityLevel, String consumerName, boolean isDurable, TransportCnx cnx, String appId, @@ -223,12 +230,8 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo stats.metadata = this.metadata; if (Subscription.isIndividualAckMode(subType)) { - this.pendingAcks = ConcurrentLongLongPairHashMap.newBuilder() - .autoShrink(subscription.getTopic().getBrokerService() - .getPulsar().getConfiguration().isAutoShrinkForConsumerPendingAcksMap()) - .expectedItems(256) - .concurrencyLevel(1) - .build(); + this.pendingAcks = new PendingAcksMap(this, this::getPendingAcksAddHandler, + this::getPendingAcksRemoveHandler); } else { // We don't need to keep track of pending acks if the subscription is not shared this.pendingAcks = null; @@ -359,17 +362,43 @@ public Future sendMessages(final List entries, // because this consumer is possible to disconnect at this time. if (pendingAcks != null) { int batchSize = batchSizes.getBatchSize(i); - int stickyKeyHash = stickyKeyHashes == null ? getStickyKeyHash(entry) : stickyKeyHashes.get(i); - long[] ackSet = batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i); - if (ackSet != null) { - unackedMessages -= (batchSize - BitSet.valueOf(ackSet).cardinality()); + int stickyKeyHash; + if (stickyKeyHashes == null) { + if (entry instanceof EntryAndMetadata entryAndMetadata) { + stickyKeyHash = entryAndMetadata.getCachedStickyKeyHash(); + } else { + stickyKeyHash = STICKY_KEY_HASH_NOT_SET; + } + } else { + stickyKeyHash = stickyKeyHashes.get(i); } - pendingAcks.put(entry.getLedgerId(), entry.getEntryId(), batchSize, stickyKeyHash); - if (log.isDebugEnabled()) { - log.debug("[{}-{}] Added {}:{} ledger entry with batchSize of {} to pendingAcks in" - + " broker.service.Consumer for consumerId: {}", - topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize, - consumerId); + boolean sendingAllowed = + pendingAcks.addPendingAckIfAllowed(entry.getLedgerId(), entry.getEntryId(), batchSize, + stickyKeyHash); + if (!sendingAllowed) { + // sending isn't allowed when pending acks doesn't accept adding the entry + // this happens when Key_Shared draining hashes contains the stickyKeyHash + // because of race conditions, it might be resolved at the time of sending + totalEntries--; + entries.set(i, null); + entry.release(); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Skipping sending of {}:{} ledger entry with batchSize of {} since adding" + + " to pending acks failed in broker.service.Consumer for consumerId: {}", + topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize, + consumerId); + } + } else { + long[] ackSet = batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i); + if (ackSet != null) { + unackedMessages -= (batchSize - BitSet.valueOf(ackSet).cardinality()); + } + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Added {}:{} ledger entry with batchSize of {} to pendingAcks in" + + " broker.service.Consumer for consumerId: {}", + topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize, + consumerId); + } } } } @@ -537,11 +566,11 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map ackOwnerConsumerAndBatchSize = + ObjectIntPair ackOwnerConsumerAndBatchSize = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), msgId.getEntryId()); - Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.getLeft(); + Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.left(); long ackedCount; - long batchSize = ackOwnerConsumerAndBatchSize.getRight(); + int batchSize = ackOwnerConsumerAndBatchSize.rightInt(); if (msgId.getAckSetsCount() > 0) { long[] ackSets = new long[msgId.getAckSetsCount()]; for (int j = 0; j < msgId.getAckSetsCount(); j++) { @@ -607,11 +636,17 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { for (int i = 0; i < ack.getMessageIdsCount(); i++) { MessageIdData msgId = ack.getMessageIdAt(i); Position position = AckSetStateUtil.createPositionWithAckSet(msgId.getLedgerId(), msgId.getEntryId(), null); - Consumer ackOwnerConsumer = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), - msgId.getEntryId()).getLeft(); + ObjectIntPair ackOwnerConsumerAndBatchSize = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), + msgId.getEntryId()); + if (ackOwnerConsumerAndBatchSize == null) { + log.warn("[{}] [{}] Acknowledging message at {} that was already deleted", subscription, + consumerId, position); + continue; + } + Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.left(); // acked count at least one long ackedCount; - long batchSize; + int batchSize; if (msgId.hasBatchSize()) { batchSize = msgId.getBatchSize(); // ack batch messages set ackeCount = batchSize @@ -660,7 +695,7 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { return completableFuture.thenApply(__ -> totalAckCount.sum()); } - private long getAckedCountForMsgIdNoAckSets(long batchSize, Position position, Consumer consumer) { + private long getAckedCountForMsgIdNoAckSets(int batchSize, Position position, Consumer consumer) { if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType)) { long[] cursorAckSet = getCursorAckSet(position); if (cursorAckSet != null) { @@ -670,11 +705,11 @@ private long getAckedCountForMsgIdNoAckSets(long batchSize, Position position, C return batchSize; } - private long getAckedCountForBatchIndexLevelEnabled(Position position, long batchSize, long[] ackSets, + private long getAckedCountForBatchIndexLevelEnabled(Position position, int batchSize, long[] ackSets, Consumer consumer) { long ackedCount = 0; if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType) - && consumer.getPendingAcks().get(position.getLedgerId(), position.getEntryId()) != null) { + && consumer.getPendingAcks().contains(position.getLedgerId(), position.getEntryId())) { long[] cursorAckSet = getCursorAckSet(position); if (cursorAckSet != null) { BitSetRecyclable cursorBitSet = BitSetRecyclable.create().resetWords(cursorAckSet); @@ -692,14 +727,14 @@ private long getAckedCountForBatchIndexLevelEnabled(Position position, long batc return ackedCount; } - private long getAckedCountForTransactionAck(long batchSize, long[] ackSets) { + private long getAckedCountForTransactionAck(int batchSize, long[] ackSets) { BitSetRecyclable bitset = BitSetRecyclable.create().resetWords(ackSets); long ackedCount = batchSize - bitset.cardinality(); bitset.recycle(); return ackedCount; } - private long getUnAckedCountForBatchIndexLevelEnabled(Position position, long batchSize) { + private long getUnAckedCountForBatchIndexLevelEnabled(Position position, int batchSize) { long unAckedCount = batchSize; if (isAcknowledgmentAtBatchIndexLevelEnabled) { long[] cursorAckSet = getCursorAckSet(position); @@ -734,24 +769,24 @@ private boolean checkCanRemovePendingAcksAndHandle(Consumer ackOwnedConsumer, * @param entryId The ID of the entry. * @return Pair */ - private Pair getAckOwnerConsumerAndBatchSize(long ledgerId, long entryId) { + private ObjectIntPair getAckOwnerConsumerAndBatchSize(long ledgerId, long entryId) { if (Subscription.isIndividualAckMode(subType)) { - LongPair longPair = getPendingAcks().get(ledgerId, entryId); - if (longPair != null) { - return Pair.of(this, longPair.first); + IntIntPair pendingAck = getPendingAcks().get(ledgerId, entryId); + if (pendingAck != null) { + return ObjectIntPair.of(this, pendingAck.leftInt()); } else { // If there are more consumers, this step will consume more CPU, and it should be optimized later. for (Consumer consumer : subscription.getConsumers()) { if (consumer != this) { - longPair = consumer.getPendingAcks().get(ledgerId, entryId); - if (longPair != null) { - return Pair.of(consumer, longPair.first); + pendingAck = consumer.getPendingAcks().get(ledgerId, entryId); + if (pendingAck != null) { + return ObjectIntPair.of(consumer, pendingAck.leftInt()); } } } } } - return Pair.of(this, 1L); + return ObjectIntPair.of(this, 1); } private long[] getCursorAckSet(Position position) { @@ -1027,7 +1062,8 @@ public int hashCode() { * @param position */ private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position) { - if (!ackOwnedConsumer.getPendingAcks().remove(position.getLedgerId(), position.getEntryId())) { + PendingAcksMap ownedConsumerPendingAcks = ackOwnedConsumer.getPendingAcks(); + if (!ownedConsumerPendingAcks.remove(position.getLedgerId(), position.getEntryId())) { // Message was already removed by the other consumer return false; } @@ -1046,7 +1082,7 @@ private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position) return true; } - public ConcurrentLongLongPairHashMap getPendingAcks() { + public PendingAcksMap getPendingAcks() { return pendingAcks; } @@ -1093,9 +1129,9 @@ public void redeliverUnacknowledgedMessages(List messageIds) { List pendingPositions = new ArrayList<>(); for (MessageIdData msg : messageIds) { Position position = PositionFactory.create(msg.getLedgerId(), msg.getEntryId()); - LongPair longPair = pendingAcks.get(position.getLedgerId(), position.getEntryId()); - if (longPair != null) { - int unAckedCount = (int) getUnAckedCountForBatchIndexLevelEnabled(position, longPair.first); + IntIntPair pendingAck = pendingAcks.get(position.getLedgerId(), position.getEntryId()); + if (pendingAck != null) { + int unAckedCount = (int) getUnAckedCountForBatchIndexLevelEnabled(position, pendingAck.leftInt()); pendingAcks.remove(position.getLedgerId(), position.getEntryId()); totalRedeliveryMessages += unAckedCount; pendingPositions.add(position); @@ -1191,16 +1227,6 @@ public Map getMetadata() { return metadata; } - private int getStickyKeyHash(Entry entry) { - final byte[] stickyKey; - if (entry instanceof EntryAndMetadata) { - stickyKey = ((EntryAndMetadata) entry).getStickyKey(); - } else { - stickyKey = Commands.peekStickyKey(entry.getDataBuffer(), topicName, subscription.getName()); - } - return StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey); - } - private static final Logger log = LoggerFactory.getLogger(Consumer.class); public Attributes getOpenTelemetryAttributes() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java new file mode 100644 index 0000000000000..d2bd113e69d1e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import com.google.common.annotations.VisibleForTesting; +import java.util.ArrayList; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; +import lombok.EqualsAndHashCode; +import lombok.ToString; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.client.api.Range; +import org.jetbrains.annotations.NotNull; + +/** + * Represents the hash ranges assigned to each consumer in a {@link StickyKeyConsumerSelector} at a point in time. + */ +@EqualsAndHashCode(exclude = "cachedRangesByConsumer") +@ToString(exclude = "cachedRangesByConsumer") +public class ConsumerHashAssignmentsSnapshot { + private final List hashRangeAssignments; + private Map> cachedRangesByConsumer; + + private ConsumerHashAssignmentsSnapshot(List hashRangeAssignments) { + validate(hashRangeAssignments); + this.hashRangeAssignments = hashRangeAssignments; + } + + private void validate(List hashRangeAssignments) { + Range previousRange = null; + for (HashRangeAssignment hashRangeAssignment : hashRangeAssignments) { + Range range = hashRangeAssignment.range(); + Consumer consumer = hashRangeAssignment.consumer(); + if (range == null || consumer == null) { + throw new IllegalArgumentException("Range and consumer must not be null"); + } + if (previousRange != null && previousRange.compareTo(range) >= 0) { + throw new IllegalArgumentException("Ranges must be non-overlapping and sorted"); + } + previousRange = range; + } + } + + public static ConsumerHashAssignmentsSnapshot of(List hashRangeAssignments) { + return new ConsumerHashAssignmentsSnapshot(hashRangeAssignments); + } + + public static ConsumerHashAssignmentsSnapshot empty() { + return new ConsumerHashAssignmentsSnapshot(Collections.emptyList()); + } + + public ImpactedConsumersResult resolveImpactedConsumers(ConsumerHashAssignmentsSnapshot other) { + return resolveConsumerRemovedHashRanges(this.hashRangeAssignments, other.hashRangeAssignments); + } + + /** + * Get the ranges assigned to each consumer. The ranges are merged if they are overlapping. + * @return the ranges assigned to each consumer + */ + public synchronized Map> getRangesByConsumer() { + if (cachedRangesByConsumer == null) { + cachedRangesByConsumer = internalGetRangesByConsumer(); + } + return cachedRangesByConsumer; + } + + private @NotNull Map> internalGetRangesByConsumer() { + Map> rangesByConsumer = new IdentityHashMap<>(); + hashRangeAssignments.forEach(entry -> { + Range range = entry.range(); + Consumer consumer = entry.consumer(); + rangesByConsumer.computeIfAbsent(consumer, k -> new TreeSet<>()).add(range); + }); + Map> mergedOverlappingRangesByConsumer = new IdentityHashMap<>(); + rangesByConsumer.forEach((consumer, ranges) -> { + mergedOverlappingRangesByConsumer.put(consumer, mergeOverlappingRanges(ranges)); + }); + return mergedOverlappingRangesByConsumer; + } + + @VisibleForTesting + Map> diffRanges(ConsumerHashAssignmentsSnapshot other) { + return diffRanges(this.hashRangeAssignments, other.hashRangeAssignments); + } + + /** + * Resolve the consumers where the existing range was removed by a change. + * @param mappingBefore the range mapping before the change + * @param mappingAfter the range mapping after the change + * @return consumers and ranges where the existing range changed + */ + static ImpactedConsumersResult resolveConsumerRemovedHashRanges(List mappingBefore, + List mappingAfter) { + Map> impactedRanges = diffRanges(mappingBefore, mappingAfter); + Map> removedRangesByConsumer = impactedRanges.entrySet().stream() + .collect(IdentityHashMap::new, (resultMap, entry) -> { + Range range = entry.getKey(); + // filter out only where the range was removed + Consumer consumerBefore = entry.getValue().getLeft(); + if (consumerBefore != null) { + resultMap.computeIfAbsent(consumerBefore, k -> new TreeSet<>()).add(range); + } + }, IdentityHashMap::putAll); + return mergedOverlappingRangesAndConvertToImpactedConsumersResult(removedRangesByConsumer); + } + + static ImpactedConsumersResult mergedOverlappingRangesAndConvertToImpactedConsumersResult( + Map> removedRangesByConsumer) { + Map mergedRangesByConsumer = new IdentityHashMap<>(); + removedRangesByConsumer.forEach((consumer, ranges) -> { + mergedRangesByConsumer.put(consumer, RemovedHashRanges.of(mergeOverlappingRanges(ranges))); + }); + return ImpactedConsumersResult.of(mergedRangesByConsumer); + } + + /** + * Merge overlapping ranges. + * @param ranges the ranges to merge + * @return the merged ranges + */ + static List mergeOverlappingRanges(SortedSet ranges) { + List mergedRanges = new ArrayList<>(); + Iterator rangeIterator = ranges.iterator(); + Range currentRange = rangeIterator.hasNext() ? rangeIterator.next() : null; + while (rangeIterator.hasNext()) { + Range nextRange = rangeIterator.next(); + if (currentRange.getEnd() >= nextRange.getStart() - 1) { + currentRange = Range.of(currentRange.getStart(), Math.max(currentRange.getEnd(), nextRange.getEnd())); + } else { + mergedRanges.add(currentRange); + currentRange = nextRange; + } + } + if (currentRange != null) { + mergedRanges.add(currentRange); + } + return mergedRanges; + } + + /** + * Calculate the diff of two range mappings. + * @param mappingBefore the range mapping before + * @param mappingAfter the range mapping after + * @return the impacted ranges where the consumer is changed from the before to the after + */ + static Map> diffRanges(List mappingBefore, + List mappingAfter) { + Map> impactedRanges = new LinkedHashMap<>(); + Iterator beforeIterator = mappingBefore.iterator(); + Iterator afterIterator = mappingAfter.iterator(); + + HashRangeAssignment beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + HashRangeAssignment afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + + while (beforeEntry != null && afterEntry != null) { + Range beforeRange = beforeEntry.range(); + Range afterRange = afterEntry.range(); + Consumer beforeConsumer = beforeEntry.consumer(); + Consumer afterConsumer = afterEntry.consumer(); + + if (beforeRange.equals(afterRange)) { + if (!beforeConsumer.equals(afterConsumer)) { + impactedRanges.put(afterRange, Pair.of(beforeConsumer, afterConsumer)); + } + beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + } else if (beforeRange.getEnd() < afterRange.getStart()) { + impactedRanges.put(beforeRange, Pair.of(beforeConsumer, afterConsumer)); + beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + } else if (afterRange.getEnd() < beforeRange.getStart()) { + impactedRanges.put(afterRange, Pair.of(beforeConsumer, afterConsumer)); + afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + } else { + Range overlapRange = Range.of( + Math.max(beforeRange.getStart(), afterRange.getStart()), + Math.min(beforeRange.getEnd(), afterRange.getEnd()) + ); + if (!beforeConsumer.equals(afterConsumer)) { + impactedRanges.put(overlapRange, Pair.of(beforeConsumer, afterConsumer)); + } + if (beforeRange.getEnd() <= overlapRange.getEnd()) { + beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + } + if (afterRange.getEnd() <= overlapRange.getEnd()) { + afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + } + } + } + + while (beforeEntry != null) { + impactedRanges.put(beforeEntry.range(), Pair.of(beforeEntry.consumer(), null)); + beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + } + + while (afterEntry != null) { + impactedRanges.put(afterEntry.range(), Pair.of(null, afterEntry.consumer())); + afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + } + + return impactedRanges; + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java new file mode 100644 index 0000000000000..3521fa197a13d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import lombok.ToString; +import lombok.extern.slf4j.Slf4j; + +/** + * A thread-safe map to store draining hashes in the consumer. + */ +@Slf4j +public class DrainingHashesTracker { + private final String dispatcherName; + private final UnblockingHandler unblockingHandler; + // optimize the memory consumption of the map by using primitive int keys + private final Int2ObjectOpenHashMap drainingHashes = new Int2ObjectOpenHashMap<>(); + int batchLevel; + boolean unblockedWhileBatching; + + /** + * Represents an entry in the draining hashes tracker. + */ + @ToString + public static class DrainingHashEntry { + private final Consumer consumer; + private int refCount; + private int blockedCount; + + /** + * Constructs a new DrainingHashEntry with the specified Consumer. + * + * @param consumer the Consumer instance + */ + DrainingHashEntry(Consumer consumer) { + this.consumer = consumer; + } + + /** + * Gets the consumer that contained the hash in pending acks at the time of creating this + * entry. Since a particular hash can be assigned to only one consumer at a time, this consumer + * cannot change. No new pending acks can be added in the {@link PendingAcksMap} when there's + * a draining hash entry for a hash in {@link DrainingHashesTracker}. + * + * @return the consumer instance that contained the hash in pending acks at the time of creating this entry + */ + public Consumer getConsumer() { + return consumer; + } + + /** + * Increments the reference count. + */ + void incrementRefCount() { + refCount++; + } + + /** + * Decrements the reference count. + * + * @return true if the reference count is zero, false otherwise + */ + boolean decrementRefCount() { + return --refCount == 0; + } + + /** + * Increments the blocked count. + */ + void incrementBlockedCount() { + blockedCount++; + } + + /** + * Checks if the entry is blocking. + * + * @return true if the blocked count is greater than zero, false otherwise + */ + boolean isBlocking() { + return blockedCount > 0; + } + } + + /** + * Interface for handling the unblocking of sticky key hashes. + */ + public interface UnblockingHandler { + /** + * Handle the unblocking of a sticky key hash. + * + * @param stickyKeyHash the sticky key hash that has been unblocked, or -1 if hash unblocking is done in batch + */ + void stickyKeyHashUnblocked(int stickyKeyHash); + } + + public DrainingHashesTracker(String dispatcherName, UnblockingHandler unblockingHandler) { + this.dispatcherName = dispatcherName; + this.unblockingHandler = unblockingHandler; + } + + /** + * Add an entry to the draining hashes tracker. + * + * @param consumer the consumer + * @param stickyHash the sticky hash + */ + public synchronized void addEntry(Consumer consumer, int stickyHash) { + if (stickyHash == 0) { + throw new IllegalArgumentException("Sticky hash cannot be 0"); + } + DrainingHashEntry entry = drainingHashes.get(stickyHash); + if (entry == null) { + entry = new DrainingHashEntry(consumer); + drainingHashes.put(stickyHash, entry); + } else if (entry.getConsumer() != consumer) { + throw new IllegalStateException( + "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash + + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer + + "."); + } + entry.incrementRefCount(); + } + + /** + * Start a batch operation. There could be multiple nested batch operations. + * The unblocking of sticky key hashes will be done only when the last batch operation ends. + */ + public synchronized void startBatch() { + batchLevel++; + } + + /** + * End a batch operation. + */ + public synchronized void endBatch() { + if (--batchLevel == 0 && unblockedWhileBatching) { + unblockedWhileBatching = false; + unblockingHandler.stickyKeyHashUnblocked(-1); + } + } + + /** + * Reduce the reference count for a given sticky hash. + * + * @param consumer the consumer + * @param stickyHash the sticky hash + * @param closing + */ + public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boolean closing) { + if (stickyHash == 0) { + return; + } + DrainingHashEntry entry = drainingHashes.get(stickyHash); + if (entry == null) { + return; + } + if (entry.getConsumer() != consumer) { + throw new IllegalStateException( + "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash + + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer + + "."); + } + if (entry.decrementRefCount()) { + DrainingHashEntry removed = drainingHashes.remove(stickyHash); + if (!closing && removed.isBlocking()) { + if (batchLevel > 0) { + unblockedWhileBatching = true; + } else { + unblockingHandler.stickyKeyHashUnblocked(stickyHash); + } + } + } + } + + /** + * Check if a sticky key hash should be blocked. + * + * @param consumer the consumer + * @param stickyKeyHash the sticky key hash + * @return true if the sticky key hash should be blocked, false otherwise + */ + public synchronized boolean shouldBlockStickyKeyHash(Consumer consumer, int stickyKeyHash) { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + log.warn("[{}] Sticky key hash is not set. Allowing dispatching", dispatcherName); + return false; + } + DrainingHashEntry entry = drainingHashes.get(stickyKeyHash); + // if the entry is not found, the hash is not draining. Don't block the hash. + if (entry == null) { + return false; + } + // hash has been reassigned to the original consumer, remove the entry + // and don't block the hash + if (entry.getConsumer() == consumer) { + log.info("[{}] Hash {} has been reassigned consumer {}. " + + "The draining hash entry with refCount={} will be removed.", + dispatcherName, stickyKeyHash, entry.getConsumer(), entry.refCount); + drainingHashes.remove(stickyKeyHash, entry); + return false; + } + // increment the blocked count which is used to determine if the hash is blocking + // dispatching to other consumers + entry.incrementBlockedCount(); + // block the hash + return true; + } + + /** + * Get the entry for a given sticky key hash. + * + * @param stickyKeyHash the sticky key hash + * @return the draining hash entry, or null if not found + */ + public synchronized DrainingHashEntry getEntry(int stickyKeyHash) { + return stickyKeyHash != 0 ? drainingHashes.get(stickyKeyHash) : null; + } + + /** + * Clear all entries in the draining hashes tracker. + */ + public synchronized void clear() { + drainingHashes.clear(); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java index efa89a8ff16f6..e4870bf251ecb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java @@ -20,7 +20,7 @@ import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; -import java.nio.charset.StandardCharsets; +import java.util.function.ToIntFunction; import javax.annotation.Nullable; import lombok.Getter; import org.apache.bookkeeper.mledger.Entry; @@ -29,11 +29,12 @@ import org.apache.pulsar.common.protocol.Commands; public class EntryAndMetadata implements Entry { - + private static final int STICKY_KEY_HASH_NOT_INITIALIZED = -1; private final Entry entry; @Getter @Nullable private final MessageMetadata metadata; + int stickyKeyHash = STICKY_KEY_HASH_NOT_INITIALIZED; private EntryAndMetadata(final Entry entry, @Nullable final MessageMetadata metadata) { this.entry = entry; @@ -45,22 +46,15 @@ public static EntryAndMetadata create(final Entry entry, final MessageMetadata m } @VisibleForTesting - static EntryAndMetadata create(final Entry entry) { + public static EntryAndMetadata create(final Entry entry) { return create(entry, Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), "", -1)); } public byte[] getStickyKey() { if (metadata != null) { - if (metadata.hasOrderingKey()) { - return metadata.getOrderingKey(); - } else if (metadata.hasPartitionKey()) { - return metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); - } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { - String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); - return fallbackKey.getBytes(StandardCharsets.UTF_8); - } + return Commands.resolveStickyKey(metadata); } - return "NONE_KEY".getBytes(StandardCharsets.UTF_8); + return Commands.NONE_KEY; } @Override @@ -114,4 +108,32 @@ public long getEntryId() { public boolean release() { return entry.release(); } + + /** + * Get cached sticky key hash or calculate it based on the sticky key if it's not cached. + * + * @param makeStickyKeyHash function to calculate the sticky key hash + * @return the sticky key hash + */ + public int getOrUpdateCachedStickyKeyHash(ToIntFunction makeStickyKeyHash) { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_INITIALIZED) { + stickyKeyHash = makeStickyKeyHash.applyAsInt(getStickyKey()); + } + return stickyKeyHash; + } + + /** + * Get cached sticky key hash or return STICKY_KEY_HASH_NOT_SET if it's not cached. + * + * @return the cached sticky key hash or STICKY_KEY_HASH_NOT_SET if it's not cached + */ + public int getCachedStickyKeyHash() { + return stickyKeyHash != STICKY_KEY_HASH_NOT_INITIALIZED ? stickyKeyHash + : StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; + } + + @VisibleForTesting + public Entry unwrap() { + return entry; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java new file mode 100644 index 0000000000000..80996c395e352 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import org.apache.pulsar.client.api.Range; + +/** + * Hash range assignment in {@link StickyKeyConsumerSelector} implementations. + */ +public record HashRangeAssignment(Range range, Consumer consumer) {} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java index a9fea5b39bf82..b90aef739f2b1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java @@ -27,7 +27,6 @@ import java.util.concurrent.ConcurrentSkipListMap; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; import org.apache.pulsar.client.api.Range; -import org.apache.pulsar.common.util.FutureUtil; /** * This is a consumer selector based fixed hash range. @@ -56,11 +55,11 @@ * */ public class HashRangeAutoSplitStickyKeyConsumerSelector implements StickyKeyConsumerSelector { - private final int rangeSize; - + private final Range keyHashRange; private final ConcurrentSkipListMap rangeMap; private final Map consumerRange; + private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public HashRangeAutoSplitStickyKeyConsumerSelector() { this(DEFAULT_RANGE_SIZE); @@ -76,10 +75,12 @@ public HashRangeAutoSplitStickyKeyConsumerSelector(int rangeSize) { this.rangeMap = new ConcurrentSkipListMap<>(); this.consumerRange = new HashMap<>(); this.rangeSize = rangeSize; + this.keyHashRange = Range.of(0, rangeSize - 1); + this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); } @Override - public synchronized CompletableFuture addConsumer(Consumer consumer) { + public synchronized CompletableFuture addConsumer(Consumer consumer) { if (rangeMap.isEmpty()) { rangeMap.put(rangeSize, consumer); consumerRange.put(consumer, rangeSize); @@ -87,14 +88,18 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { try { splitRange(findBiggestRange(), consumer); } catch (ConsumerAssignException e) { - return FutureUtil.failedFuture(e); + return CompletableFuture.failedFuture(e); } } - return CompletableFuture.completedFuture(null); + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return CompletableFuture.completedFuture(impactedConsumers); } @Override - public synchronized void removeConsumer(Consumer consumer) { + public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { Integer removeRange = consumerRange.remove(consumer); if (removeRange != null) { if (removeRange == rangeSize && rangeMap.size() > 1) { @@ -106,28 +111,40 @@ public synchronized void removeConsumer(Consumer consumer) { rangeMap.remove(removeRange); } } + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return impactedConsumers; } @Override public Consumer select(int hash) { if (!rangeMap.isEmpty()) { - int slot = hash % rangeSize; - return rangeMap.ceilingEntry(slot).getValue(); + return rangeMap.ceilingEntry(hash).getValue(); } else { return null; } } @Override - public Map> getConsumerKeyHashRanges() { - Map> result = new HashMap<>(); + public Range getKeyHashRange() { + return keyHashRange; + } + + @Override + public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { + return consumerHashAssignmentsSnapshot; + } + + private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { + List result = new ArrayList<>(); int start = 0; - for (Map.Entry entry: rangeMap.entrySet()) { - result.computeIfAbsent(entry.getValue(), key -> new ArrayList<>()) - .add(Range.of(start, entry.getKey())); + for (Entry entry: rangeMap.entrySet()) { + result.add(new HashRangeAssignment(Range.of(start, entry.getKey()), entry.getValue())); start = entry.getKey() + 1; } - return result; + return ConsumerHashAssignmentsSnapshot.of(result); } private int findBiggestRange() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java index 78bad1b2c400e..7c76d9dca7456 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -36,9 +35,10 @@ * else there'll be chance that a key fall in a `whole` that not handled by any consumer. */ public class HashRangeExclusiveStickyKeyConsumerSelector implements StickyKeyConsumerSelector { - private final int rangeSize; + private final Range keyHashRange; private final ConcurrentSkipListMap rangeMap; + private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public HashRangeExclusiveStickyKeyConsumerSelector() { this(DEFAULT_RANGE_SIZE); @@ -50,21 +50,23 @@ public HashRangeExclusiveStickyKeyConsumerSelector(int rangeSize) { throw new IllegalArgumentException("range size must greater than 0"); } this.rangeSize = rangeSize; + this.keyHashRange = Range.of(0, rangeSize - 1); this.rangeMap = new ConcurrentSkipListMap<>(); + this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); } @Override - public synchronized CompletableFuture addConsumer(Consumer consumer) { - return validateKeySharedMeta(consumer).thenRun(() -> { + public synchronized CompletableFuture addConsumer(Consumer consumer) { + return validateKeySharedMeta(consumer).thenApply(__ -> { try { - internalAddConsumer(consumer); + return internalAddConsumer(consumer); } catch (BrokerServiceException.ConsumerAssignException e) { throw FutureUtil.wrapToCompletionException(e); } }); } - private synchronized void internalAddConsumer(Consumer consumer) + private synchronized ImpactedConsumersResult internalAddConsumer(Consumer consumer) throws BrokerServiceException.ConsumerAssignException { Consumer conflictingConsumer = findConflictingConsumer(consumer.getKeySharedMeta().getHashRangesList()); if (conflictingConsumer != null) { @@ -75,37 +77,49 @@ private synchronized void internalAddConsumer(Consumer consumer) rangeMap.put(intRange.getStart(), consumer); rangeMap.put(intRange.getEnd(), consumer); } + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return impactedConsumers; } @Override - public void removeConsumer(Consumer consumer) { + public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { rangeMap.entrySet().removeIf(entry -> entry.getValue().equals(consumer)); + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return impactedConsumers; } @Override - public Map> getConsumerKeyHashRanges() { - Map> result = new HashMap<>(); + public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { + return consumerHashAssignmentsSnapshot; + } + + private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { + List result = new ArrayList<>(); Map.Entry prev = null; for (Map.Entry entry: rangeMap.entrySet()) { if (prev == null) { prev = entry; } else { if (prev.getValue().equals(entry.getValue())) { - result.computeIfAbsent(entry.getValue(), key -> new ArrayList<>()) - .add(Range.of(prev.getKey(), entry.getKey())); + result.add(new HashRangeAssignment(Range.of(prev.getKey(), entry.getKey()), entry.getValue())); } prev = null; } } - return result; + return ConsumerHashAssignmentsSnapshot.of(result); } @Override public Consumer select(int hash) { if (rangeMap.size() > 0) { - int slot = hash % rangeSize; - Map.Entry ceilingEntry = rangeMap.ceilingEntry(slot); - Map.Entry floorEntry = rangeMap.floorEntry(slot); + Map.Entry ceilingEntry = rangeMap.ceilingEntry(hash); + Map.Entry floorEntry = rangeMap.floorEntry(hash); Consumer ceilingConsumer = ceilingEntry != null ? ceilingEntry.getValue() : null; Consumer floorConsumer = floorEntry != null ? floorEntry.getValue() : null; if (floorConsumer != null && floorConsumer.equals(ceilingConsumer)) { @@ -173,4 +187,8 @@ Map getRangeConsumer() { return Collections.unmodifiableMap(rangeMap); } + @Override + public Range getKeyHashRange() { + return keyHashRange; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java new file mode 100644 index 0000000000000..a525b0501d767 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Map; +import lombok.EqualsAndHashCode; +import lombok.ToString; + +/** + * Represents the consumers that were impacted by a hash range change in a {@link StickyKeyConsumerSelector} + * at a point in time. + */ +@EqualsAndHashCode +@ToString +public class ImpactedConsumersResult { + public interface RemovedHashRangesProcessor { + void process(Consumer consumer, RemovedHashRanges removedHashRanges); + } + + private final Map removedHashRanges; + + private ImpactedConsumersResult(Map removedHashRanges) { + this.removedHashRanges = removedHashRanges; + } + + public static ImpactedConsumersResult of(Map removedHashRanges) { + return new ImpactedConsumersResult(removedHashRanges); + } + + public void processRemovedHashRanges(RemovedHashRangesProcessor processor) { + removedHashRanges.forEach((c, r) -> processor.process(c, r)); + } + + public boolean isEmpty() { + return removedHashRanges.isEmpty(); + } + + @VisibleForTesting + Map getRemovedHashRanges() { + return removedHashRanges; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java new file mode 100644 index 0000000000000..7a728a037dc62 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java @@ -0,0 +1,424 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import it.unimi.dsi.fastutil.ints.IntIntPair; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectRBTreeMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectSortedMap; +import it.unimi.dsi.fastutil.objects.ObjectBidirectionalIterator; +import java.util.Map; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Supplier; + +/** + * A thread-safe map to store pending acks in the consumer. + * + * The locking solution is used for the draining hashes solution + * to ensure that there's a consistent view of the pending acks. This is needed in the DrainingHashesTracker + * to ensure that the reference counts are consistent at all times. + * Calling forEachAndClose will ensure that no more entries can be added, + * therefore no other thread cannot send out entries while the forEachAndClose is being called. + * remove is also locked to ensure that there aren't races in the removal of entries while forEachAndClose is + * running. + */ +public class PendingAcksMap { + /** + * Callback interface for handling the addition of pending acknowledgments. + */ + public interface PendingAcksAddHandler { + /** + * Handle the addition of a pending acknowledgment. + * + * @param consumer the consumer + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param stickyKeyHash the sticky key hash + * @return true if the addition is allowed, false otherwise + */ + boolean handleAdding(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash); + } + + /** + * Callback interface for handling the removal of pending acknowledgments. + */ + public interface PendingAcksRemoveHandler { + /** + * Handle the removal of a pending acknowledgment. + * + * @param consumer the consumer + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param stickyKeyHash the sticky key hash + * @param closing true if the pending ack is being removed because the map is being closed, false + * otherwise + */ + void handleRemoving(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash, boolean closing); + /** + * Start a batch of pending acknowledgment removals. + */ + void startBatch(); + /** + * End a batch of pending acknowledgment removals. + */ + void endBatch(); + } + + /** + * Callback interface for processing pending acknowledgments. + */ + public interface PendingAcksConsumer { + /** + * Accept a pending acknowledgment. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param batchSize the batch size + * @param stickyKeyHash the sticky key hash + */ + void accept(long ledgerId, long entryId, int batchSize, int stickyKeyHash); + } + + private final Consumer consumer; + private final Long2ObjectSortedMap> pendingAcks; + private final Supplier pendingAcksAddHandlerSupplier; + private final Supplier pendingAcksRemoveHandlerSupplier; + private final Lock readLock; + private final Lock writeLock; + private boolean closed = false; + + PendingAcksMap(Consumer consumer, Supplier pendingAcksAddHandlerSupplier, + Supplier pendingAcksRemoveHandlerSupplier) { + this.consumer = consumer; + this.pendingAcks = new Long2ObjectRBTreeMap<>(); + this.pendingAcksAddHandlerSupplier = pendingAcksAddHandlerSupplier; + this.pendingAcksRemoveHandlerSupplier = pendingAcksRemoveHandlerSupplier; + ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + this.writeLock = readWriteLock.writeLock(); + this.readLock = readWriteLock.readLock(); + } + + /** + * Add a pending ack to the map if it's allowed to send a message with the given sticky key hash. + * If this method returns false, it means that the pending ack was not added, and it's not allowed to send a + * message. In that case, the caller should not send a message and skip the entry. + * The sending could be disallowed if the sticky key hash is blocked in the Key_Shared subscription. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param batchSize the batch size + * @param stickyKeyHash the sticky key hash + * @return true if the pending ack was added, and it's allowed to send a message, false otherwise + */ + public boolean addPendingAckIfAllowed(long ledgerId, long entryId, int batchSize, int stickyKeyHash) { + try { + writeLock.lock(); + // prevent adding sticky hash to pending acks if the PendingAcksMap has already been closed + // and there's a race condition between closing the consumer and sending new messages + if (closed) { + return false; + } + // prevent adding sticky hash to pending acks if it's already in draining hashes + // to avoid any race conditions that would break consistency + PendingAcksAddHandler pendingAcksAddHandler = pendingAcksAddHandlerSupplier.get(); + if (pendingAcksAddHandler != null + && !pendingAcksAddHandler.handleAdding(consumer, ledgerId, entryId, stickyKeyHash)) { + return false; + } + Long2ObjectSortedMap ledgerPendingAcks = + pendingAcks.computeIfAbsent(ledgerId, k -> new Long2ObjectRBTreeMap<>()); + ledgerPendingAcks.put(entryId, IntIntPair.of(batchSize, stickyKeyHash)); + return true; + } finally { + writeLock.unlock(); + } + } + + /** + * Get the size of the pending acks map. + * + * @return the size of the pending acks map + */ + public long size() { + try { + readLock.lock(); + return pendingAcks.values().stream().mapToInt(Long2ObjectSortedMap::size).sum(); + } finally { + readLock.unlock(); + } + } + + /** + * Iterate over all the pending acks and process them using the given processor. + * + * @param processor the processor to handle each pending ack + */ + public void forEach(PendingAcksConsumer processor) { + try { + readLock.lock(); + processPendingAcks(processor); + } finally { + readLock.unlock(); + } + } + + // iterate all pending acks and process them + private void processPendingAcks(PendingAcksConsumer processor) { + // this code uses for loops intentionally, don't refactor to use forEach + // iterate the outer map + for (Map.Entry> entry : pendingAcks.entrySet()) { + Long ledgerId = entry.getKey(); + Long2ObjectSortedMap ledgerPendingAcks = entry.getValue(); + // iterate the inner map + for (Map.Entry e : ledgerPendingAcks.entrySet()) { + Long entryId = e.getKey(); + IntIntPair batchSizeAndStickyKeyHash = e.getValue(); + processor.accept(ledgerId, entryId, batchSizeAndStickyKeyHash.leftInt(), + batchSizeAndStickyKeyHash.rightInt()); + } + } + } + + /** + * Iterate over all the pending acks and close the map so that no more entries can be added. + * All entries are removed. + * + * @param processor the processor to handle each pending ack + */ + public void forEachAndClose(PendingAcksConsumer processor) { + try { + writeLock.lock(); + closed = true; + PendingAcksRemoveHandler pendingAcksRemoveHandler = pendingAcksRemoveHandlerSupplier.get(); + if (pendingAcksRemoveHandler != null) { + try { + pendingAcksRemoveHandler.startBatch(); + processPendingAcks((ledgerId, entryId, batchSize, stickyKeyHash) -> { + processor.accept(ledgerId, entryId, batchSize, stickyKeyHash); + pendingAcksRemoveHandler.handleRemoving(consumer, ledgerId, entryId, stickyKeyHash, closed); + }); + } finally { + pendingAcksRemoveHandler.endBatch(); + } + } else { + processPendingAcks(processor); + } + pendingAcks.clear(); + } finally { + writeLock.unlock(); + } + } + + /** + * Check if the map contains a pending ack for the given ledger ID and entry ID. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @return true if the map contains the pending ack, false otherwise + */ + public boolean contains(long ledgerId, long entryId) { + try { + readLock.lock(); + Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId); + if (ledgerMap == null) { + return false; + } + return ledgerMap.containsKey(entryId); + } finally { + readLock.unlock(); + } + } + + /** + * Get the pending ack for the given ledger ID and entry ID. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @return the pending ack, or null if not found + */ + public IntIntPair get(long ledgerId, long entryId) { + try { + readLock.lock(); + Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId); + if (ledgerMap == null) { + return null; + } + return ledgerMap.get(entryId); + } finally { + readLock.unlock(); + } + } + + /** + * Remove the pending ack for the given ledger ID, entry ID, batch size, and sticky key hash. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param batchSize the batch size + * @param stickyKeyHash the sticky key hash + * @return true if the pending ack was removed, false otherwise + */ + public boolean remove(long ledgerId, long entryId, int batchSize, int stickyKeyHash) { + try { + writeLock.lock(); + Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId); + if (ledgerMap == null) { + return false; + } + boolean removed = ledgerMap.remove(entryId, IntIntPair.of(batchSize, stickyKeyHash)); + if (removed) { + handleRemovePendingAck(ledgerId, entryId, stickyKeyHash); + } + if (removed && ledgerMap.isEmpty()) { + pendingAcks.remove(ledgerId); + } + return removed; + } finally { + writeLock.unlock(); + } + } + + /** + * Remove the pending ack for the given ledger ID and entry ID. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @return true if the pending ack was removed, false otherwise + */ + public boolean remove(long ledgerId, long entryId) { + try { + writeLock.lock(); + Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId); + if (ledgerMap == null) { + return false; + } + IntIntPair removedEntry = ledgerMap.remove(entryId); + boolean removed = removedEntry != null; + if (removed) { + int stickyKeyHash = removedEntry.rightInt(); + handleRemovePendingAck(ledgerId, entryId, stickyKeyHash); + } + if (removed && ledgerMap.isEmpty()) { + pendingAcks.remove(ledgerId); + } + return removed; + } finally { + writeLock.unlock(); + } + } + + /** + * Remove all pending acks up to the given ledger ID and entry ID. + * + * @param markDeleteLedgerId the ledger ID up to which to remove pending acks + * @param markDeleteEntryId the entry ID up to which to remove pending acks + */ + public void removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) { + internalRemoveAllUpTo(markDeleteLedgerId, markDeleteEntryId, false); + } + + /** + * Removes all pending acknowledgments up to the specified ledger ID and entry ID. + * + * ReadWriteLock doesn't support upgrading from read lock to write lock. + * This method first checks if there's anything to remove using a read lock and if there is, exits + * and retries with a write lock to make the removals. + * + * @param markDeleteLedgerId the ledger ID up to which to remove pending acks + * @param markDeleteEntryId the entry ID up to which to remove pending acks + * @param useWriteLock true if the method should use a write lock, false otherwise + */ + private void internalRemoveAllUpTo(long markDeleteLedgerId, long markDeleteEntryId, boolean useWriteLock) { + PendingAcksRemoveHandler pendingAcksRemoveHandler = pendingAcksRemoveHandlerSupplier.get(); + // track if the write lock was acquired + boolean acquiredWriteLock = false; + // track if a batch was started + boolean batchStarted = false; + // track if the method should retry with a write lock + boolean retryWithWriteLock = false; + try { + if (useWriteLock) { + writeLock.lock(); + acquiredWriteLock = true; + } else { + readLock.lock(); + } + ObjectBidirectionalIterator>> ledgerMapIterator = + pendingAcks.headMap(markDeleteLedgerId + 1).long2ObjectEntrySet().iterator(); + while (ledgerMapIterator.hasNext()) { + Long2ObjectMap.Entry> entry = ledgerMapIterator.next(); + long ledgerId = entry.getLongKey(); + Long2ObjectSortedMap ledgerMap = entry.getValue(); + Long2ObjectSortedMap ledgerMapHead; + if (ledgerId == markDeleteLedgerId) { + ledgerMapHead = ledgerMap.headMap(markDeleteEntryId + 1); + } else { + ledgerMapHead = ledgerMap; + } + ObjectBidirectionalIterator> entryMapIterator = + ledgerMapHead.long2ObjectEntrySet().iterator(); + while (entryMapIterator.hasNext()) { + Long2ObjectMap.Entry intIntPairEntry = entryMapIterator.next(); + long entryId = intIntPairEntry.getLongKey(); + if (!acquiredWriteLock) { + retryWithWriteLock = true; + return; + } + if (pendingAcksRemoveHandler != null) { + if (!batchStarted) { + pendingAcksRemoveHandler.startBatch(); + batchStarted = true; + } + int stickyKeyHash = intIntPairEntry.getValue().rightInt(); + pendingAcksRemoveHandler.handleRemoving(consumer, ledgerId, entryId, stickyKeyHash, closed); + } + entryMapIterator.remove(); + } + if (ledgerMap.isEmpty()) { + if (!acquiredWriteLock) { + retryWithWriteLock = true; + return; + } + ledgerMapIterator.remove(); + } + } + } finally { + if (batchStarted) { + pendingAcksRemoveHandler.endBatch(); + } + if (acquiredWriteLock) { + writeLock.unlock(); + } else { + readLock.unlock(); + if (retryWithWriteLock) { + internalRemoveAllUpTo(markDeleteLedgerId, markDeleteEntryId, true); + } + } + } + } + + private void handleRemovePendingAck(long ledgerId, long entryId, int stickyKeyHash) { + PendingAcksRemoveHandler pendingAcksRemoveHandler = pendingAcksRemoveHandlerSupplier.get(); + if (pendingAcksRemoveHandler != null) { + pendingAcksRemoveHandler.handleRemoving(consumer, ledgerId, entryId, stickyKeyHash, closed); + } + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java index b4578711027ef..c39b722888f71 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java @@ -186,7 +186,7 @@ public boolean isSuccessorTo(Producer other) { && other.getEpoch() < epoch; } - public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndPayload, long batchSize, + public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndPayload, int batchSize, boolean isChunked, boolean isMarker, Position position) { if (checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize, position)) { publishMessageToTopic(headersAndPayload, sequenceId, batchSize, isChunked, isMarker, position); @@ -194,7 +194,7 @@ public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndP } public void publishMessage(long producerId, long lowestSequenceId, long highestSequenceId, - ByteBuf headersAndPayload, long batchSize, boolean isChunked, boolean isMarker, Position position) { + ByteBuf headersAndPayload, int batchSize, boolean isChunked, boolean isMarker, Position position) { if (lowestSequenceId > highestSequenceId) { cnx.execute(() -> { cnx.getCommandSender().sendSendError(producerId, highestSequenceId, ServerError.MetadataError, @@ -209,7 +209,7 @@ public void publishMessage(long producerId, long lowestSequenceId, long highestS } } - public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf headersAndPayload, long batchSize, + public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf headersAndPayload, int batchSize, Position position) { if (!isShadowTopic && position != null) { cnx.execute(() -> { @@ -267,7 +267,7 @@ public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf he return true; } - private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, long batchSize, boolean isChunked, + private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, int batchSize, boolean isChunked, boolean isMarker, Position position) { MessagePublishContext messagePublishContext = MessagePublishContext.get(this, sequenceId, headersAndPayload.readableBytes(), @@ -280,7 +280,7 @@ private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, l } private void publishMessageToTopic(ByteBuf headersAndPayload, long lowestSequenceId, long highestSequenceId, - long batchSize, boolean isChunked, boolean isMarker, Position position) { + int batchSize, boolean isChunked, boolean isMarker, Position position) { MessagePublishContext messagePublishContext = MessagePublishContext.get(this, lowestSequenceId, highestSequenceId, headersAndPayload.readableBytes(), batchSize, isChunked, System.nanoTime(), isMarker, position); @@ -375,7 +375,7 @@ private static final class MessagePublishContext implements PublishContext, Runn private long ledgerId; private long entryId; private int msgSize; - private long batchSize; + private int batchSize; private boolean chunked; private boolean isMarker; @@ -551,7 +551,7 @@ public void run() { recycle(); } - static MessagePublishContext get(Producer producer, long sequenceId, int msgSize, long batchSize, + static MessagePublishContext get(Producer producer, long sequenceId, int msgSize, int batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) { MessagePublishContext callback = RECYCLER.get(); callback.producer = producer; @@ -572,7 +572,7 @@ static MessagePublishContext get(Producer producer, long sequenceId, int msgSize } static MessagePublishContext get(Producer producer, long lowestSequenceId, long highestSequenceId, int msgSize, - long batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) { + int batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) { MessagePublishContext callback = RECYCLER.get(); callback.producer = producer; callback.sequenceId = lowestSequenceId; @@ -628,7 +628,7 @@ public void recycle() { msgSize = 0; ledgerId = -1L; entryId = -1L; - batchSize = 0L; + batchSize = 0; startTimeNs = -1L; chunked = false; isMarker = false; @@ -795,7 +795,7 @@ public void checkEncryption() { } public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, long highSequenceId, - ByteBuf headersAndPayload, long batchSize, boolean isChunked, boolean isMarker) { + ByteBuf headersAndPayload, int batchSize, boolean isChunked, boolean isMarker) { if (!checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize, null)) { return; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java new file mode 100644 index 0000000000000..1833c243f8955 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import java.util.List; +import lombok.EqualsAndHashCode; +import lombok.ToString; +import org.apache.pulsar.client.api.Range; + +/** + * Represents the hash ranges which were removed from an existing consumer by a change in the hash range assignments. + */ +@EqualsAndHashCode +@ToString +public class RemovedHashRanges { + private final Range[] sortedRanges; + + private RemovedHashRanges(List ranges) { + // Converts the set of ranges to an array to avoid iterator allocation + // when the ranges are iterator multiple times in the pending acknowledgments loop. + this.sortedRanges = ranges.toArray(new Range[0]); + validateSortedRanges(); + } + + private void validateSortedRanges() { + for (int i = 0; i < sortedRanges.length - 1; i++) { + if (sortedRanges[i].getStart() >= sortedRanges[i + 1].getStart()) { + throw new IllegalArgumentException( + "Ranges must be sorted: " + sortedRanges[i] + " and " + sortedRanges[i + 1]); + } + if (sortedRanges[i].getEnd() >= sortedRanges[i + 1].getStart()) { + throw new IllegalArgumentException( + "Ranges must not overlap: " + sortedRanges[i] + " and " + sortedRanges[i + 1]); + } + } + } + + public static RemovedHashRanges of(List ranges) { + return new RemovedHashRanges(ranges); + } + + /** + * Checks if the sticky key hash is contained in the impacted hash ranges. + */ + public boolean containsStickyKey(int stickyKeyHash) { + for (Range range : sortedRanges) { + if (range.contains(stickyKeyHash)) { + return true; + } + // Since ranges are sorted, stop checking further ranges if the start of the current range is + // greater than the stickyKeyHash. + if (range.getStart() > stickyKeyHash) { + return false; + } + } + return false; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java index e0ed75020bc82..1ead3f946c24d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java @@ -22,50 +22,94 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.api.Range; -import org.apache.pulsar.common.util.Murmur3_32Hash; +/** + * Abstraction for selecting the same consumer based on a key. + * This interface provides methods to add and remove consumers, + * select a consumer based on a sticky key or hash, and retrieve + * the hash range assignments for consumers. This is used by the Key_Shared implementation. + */ public interface StickyKeyConsumerSelector { - + /** + * The default range size used for hashing. + * This should be a power of 2 so that it's compatible with all implementations. + */ int DEFAULT_RANGE_SIZE = 2 << 15; + /** + * The value used to indicate that sticky key hash is not set. + * This value cannot be -1 since some of the data structures require non-negative values. + */ + int STICKY_KEY_HASH_NOT_SET = 0; + /** * Add a new consumer. * - * @param consumer new consumer + * @param consumer the new consumer to be added + * @return a CompletableFuture that completes with the result of impacted consumers. + * The result contains information about the existing consumers whose hash ranges were affected + * by the addition of the new consumer. */ - CompletableFuture addConsumer(Consumer consumer); + CompletableFuture addConsumer(Consumer consumer); /** * Remove the consumer. - * @param consumer consumer to be removed + * + * @param consumer the consumer to be removed + * @return the result of impacted consumers. The result contains information about the existing consumers + * whose hash ranges were affected by the removal of the consumer. */ - void removeConsumer(Consumer consumer); + ImpactedConsumersResult removeConsumer(Consumer consumer); /** * Select a consumer by sticky key. * - * @param stickyKey sticky key - * @return consumer + * @param stickyKey the sticky key to select the consumer + * @return the selected consumer */ default Consumer select(byte[] stickyKey) { return select(makeStickyKeyHash(stickyKey)); } - static int makeStickyKeyHash(byte[] stickyKey) { - return Murmur3_32Hash.getInstance().makeHash(stickyKey); + /** + * Make a hash from the sticky key. The hash value is in the range returned by the {@link #getKeyHashRange()} + * method instead of in the full range of integers. In other words, this returns the "slot". + * + * @param stickyKey the sticky key to hash + * @return the generated hash value + */ + default int makeStickyKeyHash(byte[] stickyKey) { + return StickyKeyConsumerSelectorUtils.makeStickyKeyHash(stickyKey, getKeyHashRange()); } /** * Select a consumer by hash. * - * @param hash hash corresponding to sticky key - * @return consumer + * @param hash the hash corresponding to the sticky key + * @return the selected consumer */ Consumer select(int hash); + /** + * Get the full range of hash values used by this selector. The upper bound is exclusive. + * + * @return the full range of hash values + */ + Range getKeyHashRange(); + /** * Get key hash ranges handled by each consumer. - * @return A map where key is a consumer name and value is list of hash range it receiving message for. + * + * @return a map where the key is a consumer and the value is a list of hash ranges it is receiving messages for + */ + default Map> getConsumerKeyHashRanges() { + return getConsumerHashAssignmentsSnapshot().getRangesByConsumer(); + } + + /** + * Get the current mappings of hash range to consumer. + * + * @return a snapshot of the consumer hash assignments */ - Map> getConsumerKeyHashRanges(); -} + ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot(); +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java new file mode 100644 index 0000000000000..03a107422ddbd --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.common.util.Hash; +import org.apache.pulsar.common.util.Murmur3_32Hash; + +/** + * Internal utility class for {@link StickyKeyConsumerSelector} implementations. + */ +class StickyKeyConsumerSelectorUtils { + private static final Hash HASH_INSTANCE = Murmur3_32Hash.getInstance(); + + /** + * Generates a sticky key hash from the given sticky key within the specified range. + * This method shouldn't be used by other classes than {@link StickyKeyConsumerSelector} implementations. + * To create a sticky key hash, use {@link StickyKeyConsumerSelector#makeStickyKeyHash(byte[])} instead which + * is an instance method of a {@link StickyKeyConsumerSelector}. + * + * @param stickyKey the sticky key to hash + * @param fullHashRange hash range to generate the hash value within + * @return the generated hash value, ensuring it is not zero (since zero is a special value in dispatchers) + */ + static int makeStickyKeyHash(byte[] stickyKey, Range fullHashRange) { + int hashValue = HASH_INSTANCE.makeHash(stickyKey) % fullHashRange.size() + fullHashRange.getStart(); + // Avoid using STICKY_KEY_HASH_NOT_SET as hash value + if (hashValue == STICKY_KEY_HASH_NOT_SET) { + // use next value as hash value + hashValue = STICKY_KEY_HASH_NOT_SET + 1; + } + return hashValue; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java index fb7bd22de94a7..ce674cf471ef0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java @@ -101,10 +101,10 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { consumerList.remove(consumer); } throw FutureUtil.wrapToCompletionException(ex); - } else { - return value; } - })); + return value; + })).thenAccept(__ -> { + }); } @Override @@ -152,7 +152,7 @@ public void sendMessages(List entries) { for (Entry entry : entries) { byte[] stickyKey = peekStickyKey(entry.getDataBuffer()); - int stickyKeyHash = StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey); + int stickyKeyHash = selector.makeStickyKeyHash(stickyKey); Consumer consumer = selector.select(stickyKeyHash); if (consumer != null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index fa6e1412151b6..b34a0b454385f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.broker.service.persistent; -import com.google.common.collect.ComparisonChain; +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import java.util.ArrayList; import java.util.List; import java.util.NavigableSet; @@ -65,6 +65,9 @@ public void add(long ledgerId, long entryId) { public void add(long ledgerId, long entryId, long stickyKeyHash) { if (!allowOutOfOrderDelivery) { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + throw new IllegalArgumentException("Sticky key hash is not set. It is required."); + } boolean inserted = hashesToBeBlocked.putIfAbsent(ledgerId, entryId, stickyKeyHash, 0); if (!inserted) { hashesToBeBlocked.put(ledgerId, entryId, stickyKeyHash, 0); @@ -108,18 +111,20 @@ public Long getHash(long ledgerId, long entryId) { } public void removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) { - if (!allowOutOfOrderDelivery) { + boolean bitsCleared = messagesToRedeliver.removeUpTo(markDeleteLedgerId, markDeleteEntryId + 1); + // only if bits have been clear, and we are not allowing out of order delivery, we need to remove the hashes + // removing hashes is a relatively expensive operation, so we should only do it when necessary + if (bitsCleared && !allowOutOfOrderDelivery) { List keysToRemove = new ArrayList<>(); hashesToBeBlocked.forEach((ledgerId, entryId, stickyKeyHash, none) -> { - if (ComparisonChain.start().compare(ledgerId, markDeleteLedgerId).compare(entryId, markDeleteEntryId) - .result() <= 0) { + if (ledgerId < markDeleteLedgerId || (ledgerId == markDeleteLedgerId && entryId <= markDeleteEntryId)) { keysToRemove.add(new LongPair(ledgerId, entryId)); } }); - keysToRemove.forEach(longPair -> removeFromHashBlocker(longPair.first, longPair.second)); - keysToRemove.clear(); + for (LongPair longPair : keysToRemove) { + removeFromHashBlocker(longPair.first, longPair.second); + } } - messagesToRedeliver.removeUpTo(markDeleteLedgerId, markDeleteEntryId + 1); } public boolean isEmpty() { @@ -141,7 +146,7 @@ public String toString() { public boolean containsStickyKeyHashes(Set stickyKeyHashes) { if (!allowOutOfOrderDelivery) { for (Integer stickyKeyHash : stickyKeyHashes) { - if (hashesRefCount.containsKey(stickyKeyHash)) { + if (stickyKeyHash != STICKY_KEY_HASH_NOT_SET && hashesRefCount.containsKey(stickyKeyHash)) { return true; } } @@ -150,7 +155,8 @@ public boolean containsStickyKeyHashes(Set stickyKeyHashes) { } public boolean containsStickyKeyHash(int stickyKeyHash) { - return !allowOutOfOrderDelivery && hashesRefCount.containsKey(stickyKeyHash); + return !allowOutOfOrderDelivery + && stickyKeyHash != STICKY_KEY_HASH_NOT_SET && hashesRefCount.containsKey(stickyKeyHash); } public Optional getFirstPositionInReplay() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index d479d8f384ee9..73d152bab1a60 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; @@ -46,6 +47,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; @@ -67,7 +69,6 @@ import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.SharedConsumerAssignor; -import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; import org.apache.pulsar.broker.transaction.exception.buffer.TransactionBufferException; @@ -132,6 +133,7 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul "blockedDispatcherOnUnackedMsgs"); protected Optional dispatchRateLimiter = Optional.empty(); private AtomicBoolean isRescheduleReadInProgress = new AtomicBoolean(false); + private final AtomicBoolean readMoreEntriesAsyncRequested = new AtomicBoolean(false); protected final ExecutorService dispatchMessagesThread; private final SharedConsumerAssignor assignor; // tracks how many entries were processed by consumers in the last trySendMessagesToConsumers call @@ -144,6 +146,8 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul protected enum ReadType { Normal, Replay } + private Position lastMarkDeletePositionBeforeReadMoreEntries; + private volatile long readMoreEntriesCallCount; public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor, Subscription subscription) { @@ -164,7 +168,7 @@ public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCurso : RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; this.readBatchSize = serviceConfig.getDispatcherMaxReadBatchSize(); this.initializeDispatchRateLimiterIfNeeded(); - this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addMessageToReplay); + this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addEntryToReplay); ServiceConfiguration serviceConfiguration = topic.getBrokerService().pulsar().getConfiguration(); this.readFailureBackoff = new Backoff( serviceConfiguration.getDispatcherReadFailureBackoffInitialTimeInMs(), @@ -239,8 +243,12 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE if (log.isDebugEnabled()) { log.debug("[{}] Consumer are left, reading more entries", name); } - consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { - addMessageToReplay(ledgerId, entryId, stickyKeyHash); + MutableBoolean notifyAddedToReplay = new MutableBoolean(false); + consumer.getPendingAcks().forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> { + boolean addedToReplay = addMessageToReplay(ledgerId, entryId, stickyKeyHash); + if (addedToReplay) { + notifyAddedToReplay.setTrue(); + } }); totalAvailablePermits -= consumer.getAvailablePermits(); if (log.isDebugEnabled()) { @@ -248,7 +256,9 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE + "New dispatcher permit count is {}", name, consumer.getAvailablePermits(), totalAvailablePermits); } - readMoreEntries(); + if (notifyAddedToReplay.booleanValue()) { + notifyRedeliveryMessageAdded(); + } } } else { /** @@ -264,7 +274,12 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE } } - private synchronized void clearComponentsAfterRemovedAllConsumers() { + protected synchronized void internalRemoveConsumer(Consumer consumer) { + consumerSet.removeAll(consumer); + consumerList.remove(consumer); + } + + protected synchronized void clearComponentsAfterRemovedAllConsumers() { cancelPendingRead(); redeliveryMessages.clear(); @@ -298,7 +313,7 @@ private synchronized void internalConsumerFlow(Consumer consumer, int additional + "after adding {} permits", name, consumer, totalAvailablePermits, additionalNumberOfMessages); } - readMoreEntries(); + readMoreEntriesAsync(); } /** @@ -306,7 +321,13 @@ private synchronized void internalConsumerFlow(Consumer consumer, int additional * */ public void readMoreEntriesAsync() { - topic.getBrokerService().executor().execute(this::readMoreEntries); + // deduplication for readMoreEntriesAsync calls + if (readMoreEntriesAsyncRequested.compareAndSet(false, true)) { + topic.getBrokerService().executor().execute(() -> { + readMoreEntriesAsyncRequested.set(false); + readMoreEntries(); + }); + } } public synchronized void readMoreEntries() { @@ -337,6 +358,20 @@ public synchronized void readMoreEntries() { return; } + // increment the counter for readMoreEntries calls, to track the number of times readMoreEntries is called + readMoreEntriesCallCount++; + + // remove possible expired messages from redelivery tracker and pending acks + Position markDeletePosition = cursor.getMarkDeletedPosition(); + if (lastMarkDeletePositionBeforeReadMoreEntries != markDeletePosition) { + redeliveryMessages.removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); + for (Consumer consumer : consumerList) { + consumer.getPendingAcks() + .removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); + } + lastMarkDeletePositionBeforeReadMoreEntries = markDeletePosition; + } + // totalAvailablePermits may be updated by other threads int firstAvailableConsumerPermits = getFirstAvailableConsumerPermits(); int currentTotalAvailablePermits = Math.max(totalAvailablePermits, firstAvailableConsumerPermits); @@ -396,21 +431,8 @@ public synchronized void readMoreEntries() { updateMinReplayedPosition(); messagesToRead = Math.min(messagesToRead, getMaxEntriesReadLimit()); - - // Filter out and skip read delayed messages exist in DelayedDeliveryTracker - if (delayedDeliveryTracker.isPresent()) { - Predicate skipCondition = null; - final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get(); - if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { - skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker) - .containsMessage(position.getLedgerId(), position.getEntryId()); - } - cursor.asyncReadEntriesWithSkipOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, - topic.getMaxReadPosition(), skipCondition); - } else { - cursor.asyncReadEntriesOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, - topic.getMaxReadPosition()); - } + cursor.asyncReadEntriesWithSkipOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, + topic.getMaxReadPosition(), createReadEntriesSkipConditionForNormalRead()); } else { if (log.isDebugEnabled()) { log.debug("[{}] Cannot schedule next read until previous one is done", name); @@ -423,6 +445,19 @@ public synchronized void readMoreEntries() { } } + protected Predicate createReadEntriesSkipConditionForNormalRead() { + Predicate skipCondition = null; + // Filter out and skip read delayed messages exist in DelayedDeliveryTracker + if (delayedDeliveryTracker.isPresent()) { + final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get(); + if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { + skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker) + .containsMessage(position.getLedgerId(), position.getEntryId()); + } + } + return skipCondition; + } + /** * Sets a hard limit on the number of entries to read from the Managed Ledger. * Subclasses can override this method to set a different limit. @@ -447,6 +482,10 @@ protected void handleNormalReadNotAllowed() { // do nothing } + protected long getReadMoreEntriesCallCount() { + return readMoreEntriesCallCount; + } + /** * Controls whether replaying entries is currently enabled. * Subclasses can override this method to temporarily disable replaying entries. @@ -702,7 +741,7 @@ public final synchronized void readEntriesComplete(List entries, Object c entries.forEach(Entry::release); cursor.rewind(); shouldRewindBeforeReadingOrReplaying = false; - readMoreEntries(); + readMoreEntriesAsync(); return; } @@ -799,8 +838,15 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis boolean hasChunk = false; for (int i = 0; i < metadataArray.length; i++) { Entry entry = entries.get(i); - MessageMetadata metadata = entry instanceof EntryAndMetadata ? ((EntryAndMetadata) entry).getMetadata() - : Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), subscription.toString(), -1); + MessageMetadata metadata; + if (entry instanceof EntryAndMetadata) { + metadata = ((EntryAndMetadata) entry).getMetadata(); + } else { + metadata = Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), subscription.toString(), -1); + // cache the metadata in the entry with EntryAndMetadata for later use to avoid re-parsing the metadata + // and to carry the metadata and calculated stickyKeyHash with the entry + entries.set(i, EntryAndMetadata.create(entry, metadata)); + } if (metadata != null) { remainingMessages += metadata.getNumMessagesInBatch(); if (!hasChunk && metadata.hasUuid()) { @@ -901,18 +947,17 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis log.debug("[{}] No consumers found with available permits, storing {} positions for later replay", name, entries.size() - start); } - entries.subList(start, entries.size()).forEach(entry -> { - addEntryToReplay(entry); - entry.release(); - }); + entries.subList(start, entries.size()).forEach(this::addEntryToReplay); } return true; } - protected void addEntryToReplay(Entry entry) { + protected boolean addEntryToReplay(Entry entry) { long stickyKeyHash = getStickyKeyHash(entry); - addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + boolean addedToReplay = addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + return addedToReplay; } private boolean sendChunkedMessagesToConsumers(ReadType readType, @@ -930,6 +975,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, long totalEntries = 0; long totalEntriesProcessed = 0; final AtomicInteger numConsumers = new AtomicInteger(assignResult.size()); + boolean notifyAddedToReplay = false; for (Map.Entry> current : assignResult.entrySet()) { final Consumer consumer = current.getKey(); final List entryAndMetadataList = current.getValue(); @@ -941,7 +987,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, if (messagesForC < entryAndMetadataList.size()) { for (int i = messagesForC; i < entryAndMetadataList.size(); i++) { final EntryAndMetadata entry = entryAndMetadataList.get(i); - addMessageToReplay(entry); + notifyAddedToReplay |= addEntryToReplay(entry); entryAndMetadataList.set(i, null); } } @@ -965,7 +1011,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker() ).addListener(future -> { if (future.isDone() && numConsumers.decrementAndGet() == 0) { - readMoreEntries(); + readMoreEntriesAsync(); } }); @@ -978,7 +1024,8 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); - return numConsumers.get() == 0; // trigger a new readMoreEntries() call + // trigger a new readMoreEntries() call + return numConsumers.get() == 0 || notifyAddedToReplay; } @Override @@ -1111,31 +1158,39 @@ public boolean isConsumerAvailable(Consumer consumer) { @Override public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch) { + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, + redeliveryMessages); + } + MutableBoolean addedToReplay = new MutableBoolean(false); consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { if (addMessageToReplay(ledgerId, entryId, stickyKeyHash)) { redeliveryTracker.incrementAndGetRedeliveryCount((PositionFactory.create(ledgerId, entryId))); + addedToReplay.setTrue(); } }); - if (log.isDebugEnabled()) { - log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, - redeliveryMessages); + if (addedToReplay.booleanValue()) { + notifyRedeliveryMessageAdded(); } - readMoreEntries(); } @Override public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions); + } + MutableBoolean addedToReplay = new MutableBoolean(false); positions.forEach(position -> { // TODO: We want to pass a sticky key hash as a third argument to guarantee the order of the messages // on Key_Shared subscription, but it's difficult to get the sticky key here if (addMessageToReplay(position.getLedgerId(), position.getEntryId())) { redeliveryTracker.incrementAndGetRedeliveryCount(position); + addedToReplay.setTrue(); } }); - if (log.isDebugEnabled()) { - log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions); + if (addedToReplay.booleanValue()) { + notifyRedeliveryMessageAdded(); } - readMoreEntries(); } @Override @@ -1366,11 +1421,6 @@ public void cursorIsReset() { } } - private void addMessageToReplay(Entry entry) { - addMessageToReplay(entry.getLedgerId(), entry.getEntryId()); - entry.release(); - } - protected boolean addMessageToReplay(long ledgerId, long entryId, long stickyKeyHash) { if (checkIfMessageIsUnacked(ledgerId, entryId)) { redeliveryMessages.add(ledgerId, entryId, stickyKeyHash); @@ -1380,6 +1430,13 @@ protected boolean addMessageToReplay(long ledgerId, long entryId, long stickyKey } } + /** + * Notify the dispatcher that a message has been added to the redelivery list. + */ + private void notifyRedeliveryMessageAdded() { + readMoreEntriesAsync(); + } + protected boolean addMessageToReplay(long ledgerId, long entryId) { if (checkIfMessageIsUnacked(ledgerId, entryId)) { redeliveryMessages.add(ledgerId, entryId); @@ -1404,7 +1461,7 @@ public boolean checkAndUnblockIfStuck() { if (totalAvailablePermits > 0 && !havePendingReplayRead && !havePendingRead && cursor.getNumberOfEntriesInBacklog(false) > 0) { log.warn("{}-{} Dispatcher is stuck and unblocking by issuing reads", topic.getName(), name); - readMoreEntries(); + readMoreEntriesAsync(); return true; } return false; @@ -1436,10 +1493,10 @@ public ManagedCursor getCursor() { } protected int getStickyKeyHash(Entry entry) { - return StickyKeyConsumerSelector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer())); + // There's no need to calculate the hash for Shared subscription + return STICKY_KEY_HASH_NOT_SET; } - public Subscription getSubscription() { return subscription; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index ecd3f19a14028..a78e4e46c0e5a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -18,12 +18,11 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -31,24 +30,24 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; -import javax.annotation.Nullable; +import lombok.Getter; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.ConsistentHashingStickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.DrainingHashesTracker; +import org.apache.pulsar.broker.service.EntryAndMetadata; import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; import org.apache.pulsar.broker.service.HashRangeAutoSplitStickyKeyConsumerSelector; import org.apache.pulsar.broker.service.HashRangeExclusiveStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.ImpactedConsumersResult; +import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Subscription; @@ -56,9 +55,8 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.KeySharedMeta; import org.apache.pulsar.common.api.proto.KeySharedMode; +import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,25 +64,14 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi private final boolean allowOutOfOrderDelivery; private final StickyKeyConsumerSelector selector; - private final boolean recentlyJoinedConsumerTrackingRequired; + private final boolean drainingHashesRequired; private boolean skipNextReplayToTriggerLookAhead = false; private final KeySharedMode keySharedMode; + @Getter + private final DrainingHashesTracker drainingHashesTracker; - /** - * When a consumer joins, it will be added to this map with the current read position. - * This means that, in order to preserve ordering, new consumers can only receive old - * messages, until the mark-delete position will move past this point. - */ - private final LinkedHashMap recentlyJoinedConsumers; - - /** - * The lastSentPosition and the individuallySentPositions are not thread safe. - */ - @Nullable - private Position lastSentPosition; - private final LongPairRangeSet individuallySentPositions; - private static final LongPairRangeSet.LongPairConsumer positionRangeConverter = PositionFactory::create; + private final RescheduleReadHandler rescheduleReadHandler; PersistentStickyKeyDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor, Subscription subscription, ServiceConfiguration conf, KeySharedMeta ksm) { @@ -93,13 +80,13 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi this.allowOutOfOrderDelivery = ksm.isAllowOutOfOrderDelivery(); this.keySharedMode = ksm.getKeySharedMode(); // recent joined consumer tracking is required only for AUTO_SPLIT mode when out-of-order delivery is disabled - this.recentlyJoinedConsumerTrackingRequired = + this.drainingHashesRequired = keySharedMode == KeySharedMode.AUTO_SPLIT && !allowOutOfOrderDelivery; - this.recentlyJoinedConsumers = recentlyJoinedConsumerTrackingRequired ? new LinkedHashMap<>() : null; - this.individuallySentPositions = - recentlyJoinedConsumerTrackingRequired - ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter) - : null; + this.drainingHashesTracker = + drainingHashesRequired ? new DrainingHashesTracker(this.getName(), this::stickyKeyHashUnblocked) : null; + this.rescheduleReadHandler = new RescheduleReadHandler(conf::getKeySharedUnblockingIntervalMs, + topic.getBrokerService().executor(), this::cancelPendingRead, () -> reScheduleReadInMs(0), + () -> havePendingRead, this::getReadMoreEntriesCallCount, () -> !redeliveryMessages.isEmpty()); switch (this.keySharedMode) { case AUTO_SPLIT: if (conf.isSubscriptionKeySharedUseConsistentHashing()) { @@ -109,16 +96,29 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi selector = new HashRangeAutoSplitStickyKeyConsumerSelector(); } break; - case STICKY: this.selector = new HashRangeExclusiveStickyKeyConsumerSelector(); break; - default: throw new IllegalArgumentException("Invalid key-shared mode: " + keySharedMode); } } + private void stickyKeyHashUnblocked(int stickyKeyHash) { + if (log.isDebugEnabled()) { + if (stickyKeyHash > -1) { + log.debug("[{}] Sticky key hash {} is unblocked", getName(), stickyKeyHash); + } else { + log.debug("[{}] Some sticky key hashes are unblocked", getName()); + } + } + reScheduleReadWithKeySharedUnblockingInterval(); + } + + private void reScheduleReadWithKeySharedUnblockingInterval() { + rescheduleReadHandler.rescheduleRead(); + } + @VisibleForTesting public StickyKeyConsumerSelector getSelector() { return selector; @@ -131,32 +131,52 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { consumer.disconnect(); return CompletableFuture.completedFuture(null); } - return super.addConsumer(consumer).thenCompose(__ -> - selector.addConsumer(consumer).handle((result, ex) -> { - if (ex != null) { - synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - consumerSet.removeAll(consumer); - consumerList.remove(consumer); - } - throw FutureUtil.wrapToCompletionException(ex); + return super.addConsumer(consumer).thenCompose(__ -> selector.addConsumer(consumer)) + .thenAccept(impactedConsumers -> { + // TODO: Add some way to prevent changes in between the time the consumer is added and the + // time the draining hashes are applied. It might be fine for ConsistentHashingStickyKeyConsumerSelector + // since it's not really asynchronous, although it returns a CompletableFuture + if (drainingHashesRequired) { + consumer.setPendingAcksAddHandler(this::handleAddingPendingAck); + consumer.setPendingAcksRemoveHandler(new PendingAcksMap.PendingAcksRemoveHandler() { + @Override + public void handleRemoving(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash, + boolean closing) { + drainingHashesTracker.reduceRefCount(consumer, stickyKeyHash, closing); } - return result; - }) - ).thenRun(() -> { - synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - if (recentlyJoinedConsumerTrackingRequired) { - final Position lastSentPositionWhenJoining = updateIfNeededAndGetLastSentPosition(); - if (lastSentPositionWhenJoining != null) { - consumer.setLastSentPositionWhenJoining(lastSentPositionWhenJoining); - // If this was the 1st consumer, or if all the messages are already acked, then we - // don't need to do anything special - if (recentlyJoinedConsumers != null - && consumerList.size() > 1 - && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) { - recentlyJoinedConsumers.put(consumer, lastSentPositionWhenJoining); - } + + @Override + public void startBatch() { + drainingHashesTracker.startBatch(); } - } + + @Override + public void endBatch() { + drainingHashesTracker.endBatch(); + } + }); + registerDrainingHashes(consumer, impactedConsumers); + } + }).exceptionally(ex -> { + internalRemoveConsumer(consumer); + throw FutureUtil.wrapToCompletionException(ex); + }); + } + + private synchronized void registerDrainingHashes(Consumer skipConsumer, + ImpactedConsumersResult impactedConsumers) { + impactedConsumers.processRemovedHashRanges((c, removedHashRanges) -> { + if (c != skipConsumer) { + c.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + log.warn("[{}] Sticky key hash was missing for {}:{}", getName(), ledgerId, entryId); + return; + } + if (removedHashRanges.containsStickyKey(stickyKeyHash)) { + // add the pending ack to the draining hashes tracker if the hash is in the range + drainingHashesTracker.addEntry(c, stickyKeyHash); + } + }); } }); } @@ -164,27 +184,21 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { @Override public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { // The consumer must be removed from the selector before calling the superclass removeConsumer method. - // In the superclass removeConsumer method, the pending acks that the consumer has are added to - // redeliveryMessages. If the consumer has not been removed from the selector at this point, - // the broker will try to redeliver the messages to the consumer that has already been closed. - // As a result, the messages are not redelivered to any consumer, and the mark-delete position does not move, - // eventually causing all consumers to get stuck. - selector.removeConsumer(consumer); + ImpactedConsumersResult impactedConsumers = selector.removeConsumer(consumer); super.removeConsumer(consumer); - if (recentlyJoinedConsumerTrackingRequired) { - recentlyJoinedConsumers.remove(consumer); - if (consumerList.size() == 1) { - recentlyJoinedConsumers.clear(); - } else if (consumerList.isEmpty()) { - // The subscription removes consumers if rewind or reset cursor operations are called. - // The dispatcher must clear lastSentPosition and individuallySentPositions because - // these operations trigger re-sending messages. - lastSentPosition = null; - individuallySentPositions.clear(); - } - if (removeConsumersFromRecentJoinedConsumers() || !redeliveryMessages.isEmpty()) { - readMoreEntries(); - } + if (drainingHashesRequired) { + // register draining hashes for the impacted consumers and ranges, in case a hash switched from one + // consumer to another. This will handle the case where a hash gets switched from an existing + // consumer to another existing consumer during removal. + registerDrainingHashes(consumer, impactedConsumers); + } + } + + @Override + protected synchronized void clearComponentsAfterRemovedAllConsumers() { + super.clearComponentsAfterRemovedAllConsumers(); + if (drainingHashesRequired) { + drainingHashesTracker.clear(); } } @@ -226,11 +240,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis name, replayPosition, minReplayedPosition, readType); } if (readType == ReadType.Normal) { - entries.forEach(entry -> { - long stickyKeyHash = getStickyKeyHash(entry); - addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); - entry.release(); - }); + entries.forEach(this::addEntryToReplay); } else if (readType == ReadType.Replay) { entries.forEach(Entry::release); } @@ -241,26 +251,6 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } } - if (recentlyJoinedConsumerTrackingRequired) { - // Update if the markDeletePosition move forward - updateIfNeededAndGetLastSentPosition(); - - // Should not access to individualDeletedMessages from outside managed cursor - // because it doesn't guarantee thread safety. - if (lastSentPosition == null) { - if (cursor.getMarkDeletedPosition() != null) { - lastSentPosition = ((ManagedCursorImpl) cursor) - .processIndividuallyDeletedMessagesAndGetMarkDeletedPosition(range -> { - final Position lower = range.lowerEndpoint(); - final Position upper = range.upperEndpoint(); - individuallySentPositions.addOpenClosed(lower.getLedgerId(), lower.getEntryId(), - upper.getLedgerId(), upper.getEntryId()); - return true; - }); - } - } - } - // returns a boolean indicating whether look-ahead could be useful, when there's a consumer // with available permits, and it's not able to make progress because of blocked hashes. MutableBoolean triggerLookAhead = new MutableBoolean(); @@ -276,23 +266,11 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis log.debug("[{}] select consumer {} with messages num {}, read type is {}", name, consumer.consumerName(), entriesForConsumer.size(), readType); } - final ManagedLedger managedLedger = cursor.getManagedLedger(); - for (Entry entry : entriesForConsumer) { - // remove positions first from replay list first : sendMessages recycles entries - if (readType == ReadType.Replay) { + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { + for (Entry entry : entriesForConsumer) { redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); } - // Add positions to individuallySentPositions if necessary - if (recentlyJoinedConsumerTrackingRequired) { - final Position position = entry.getPosition(); - // Store to individuallySentPositions even if lastSentPosition is null - if ((lastSentPosition == null || position.compareTo(lastSentPosition) > 0) - && !individuallySentPositions.contains(position.getLedgerId(), position.getEntryId())) { - final Position previousPosition = managedLedger.getPreviousPosition(position); - individuallySentPositions.addOpenClosed(previousPosition.getLedgerId(), - previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); - } - } } SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); @@ -306,7 +284,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker()).addListener(future -> { if (future.isDone() && remainingConsumersToFinishSending.decrementAndGet() == 0) { - readMoreEntries(); + readMoreEntriesAsync(); } }); @@ -316,60 +294,6 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis totalBytesSent += sendMessageInfo.getTotalBytes(); } - // Update the last sent position and remove ranges from individuallySentPositions if necessary - if (recentlyJoinedConsumerTrackingRequired && lastSentPosition != null) { - final ManagedLedger managedLedger = cursor.getManagedLedger(); - com.google.common.collect.Range range = individuallySentPositions.firstRange(); - - // If the upper bound is before the last sent position, we need to move ahead as these - // individuallySentPositions are now irrelevant. - if (range != null && range.upperEndpoint().compareTo(lastSentPosition) <= 0) { - individuallySentPositions.removeAtMost(lastSentPosition.getLedgerId(), - lastSentPosition.getEntryId()); - range = individuallySentPositions.firstRange(); - } - - if (range != null) { - // If the lowerBound is ahead of the last sent position, - // verify if there are any entries in-between. - if (range.lowerEndpoint().compareTo(lastSentPosition) <= 0 || managedLedger - .getNumberOfEntries(com.google.common.collect.Range.openClosed(lastSentPosition, - range.lowerEndpoint())) <= 0) { - if (log.isDebugEnabled()) { - log.debug("[{}] Found a position range to last sent: {}", name, range); - } - Position newLastSentPosition = range.upperEndpoint(); - Position positionAfterNewLastSent = managedLedger - .getNextValidPosition(newLastSentPosition); - // sometime ranges are connected but belongs to different ledgers - // so, they are placed sequentially - // eg: (2:10..3:15] can be returned as (2:10..2:15],[3:0..3:15]. - // So, try to iterate over connected range and found the last non-connected range - // which gives new last sent position. - final Position lastConfirmedEntrySnapshot = managedLedger.getLastConfirmedEntry(); - if (lastConfirmedEntrySnapshot != null) { - while (positionAfterNewLastSent.compareTo(lastConfirmedEntrySnapshot) <= 0) { - if (individuallySentPositions.contains(positionAfterNewLastSent.getLedgerId(), - positionAfterNewLastSent.getEntryId())) { - range = individuallySentPositions.rangeContaining( - positionAfterNewLastSent.getLedgerId(), positionAfterNewLastSent.getEntryId()); - newLastSentPosition = range.upperEndpoint(); - positionAfterNewLastSent = managedLedger.getNextValidPosition(newLastSentPosition); - // check if next valid position is also deleted and part of the deleted-range - continue; - } - break; - } - } - - if (lastSentPosition.compareTo(newLastSentPosition) < 0) { - lastSentPosition = newLastSentPosition; - } - individuallySentPositions.removeAtMost(lastSentPosition.getLedgerId(), - lastSentPosition.getEntryId()); - } - } - } lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; @@ -386,7 +310,8 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis // therefore would be most likely only increase the distance between read-position and mark-delete position. skipNextReplayToTriggerLookAhead = true; // skip backoff delay before reading ahead in the "look ahead" mode to prevent any additional latency - skipNextBackoff = true; + // only skip the delay if there are more entries to read + skipNextBackoff = cursor.hasMoreEntries(); return true; } @@ -398,6 +323,37 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis return false; } + /** + * Check if the sticky hash is already draining or blocked in the replay queue. + * If it is, add the message to replay and return false so that the message isn't sent to a consumer. + * + * @param ledgerId the ledger id of the message + * @param entryId the entry id of the message + * @param stickyKeyHash the sticky hash of the message + * @return true if the message should be added to pending acks and allow sending, false otherwise + */ + private boolean handleAddingPendingAck(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash) { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + log.warn("[{}] Sticky key hash is missing for {}:{}", getName(), ledgerId, entryId); + throw new IllegalArgumentException("Sticky key hash is missing for " + ledgerId + ":" + entryId); + } + DrainingHashesTracker.DrainingHashEntry drainingHashEntry = drainingHashesTracker.getEntry(stickyKeyHash); + if (drainingHashEntry != null && drainingHashEntry.getConsumer() != consumer) { + log.warn("[{}] Another consumer id {} is already draining hash {}. Skipping adding {}:{} to pending acks " + + "for consumer {}. Adding the message to replay.", + getName(), drainingHashEntry.getConsumer(), stickyKeyHash, ledgerId, entryId, consumer); + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + // block message from sending + return false; + } + if (log.isDebugEnabled()) { + log.debug("[{}] Adding {}:{} to pending acks for consumer {} with sticky key hash {}", + getName(), ledgerId, entryId, consumer, stickyKeyHash); + } + // allow adding the message to pending acks and sending the message to the consumer + return true; + } + private boolean isReplayQueueSizeBelowLimit() { return redeliveryMessages.size() < getEffectiveLookAheadLimit(); } @@ -442,16 +398,22 @@ private Map> filterAndGroupEntriesForDispatching(List> entriesGroupedByConsumer = new HashMap<>(); // permits for consumer, permits are for entries/batches Map permitsForConsumer = new HashMap<>(); - // maxLastSentPosition cache for consumers, used when recently joined consumers exist - boolean hasRecentlyJoinedConsumers = hasRecentlyJoinedConsumers(); - Map maxLastSentPositionCache = hasRecentlyJoinedConsumers ? new HashMap<>() : null; boolean lookAheadAllowed = isReplayQueueSizeBelowLimit(); // in normal read mode, keep track of consumers that are blocked by hash, to check if look-ahead could be useful Set blockedByHashConsumers = lookAheadAllowed && readType == ReadType.Normal ? new HashSet<>() : null; // in replay read mode, keep track of consumers for entries, used for look-ahead check Set consumersForEntriesForLookaheadCheck = lookAheadAllowed ? new HashSet<>() : null; - for (Entry entry : entries) { + for (Entry inputEntry : entries) { + EntryAndMetadata entry; + if (inputEntry instanceof EntryAndMetadata entryAndMetadataInstance) { + entry = entryAndMetadataInstance; + } else { + // replace the input entry with EntryAndMetadata instance. In addition to the entry and metadata, + // it will also carry the calculated sticky key hash + entry = EntryAndMetadata.create(inputEntry, + Commands.peekAndCopyMessageMetadata(inputEntry.getDataBuffer(), getSubscriptionName(), -1)); + } int stickyKeyHash = getStickyKeyHash(entry); Consumer consumer = selector.select(stickyKeyHash); MutableBoolean blockedByHash = null; @@ -460,15 +422,13 @@ private Map> filterAndGroupEntriesForDispatching(List resolveMaxLastSentPositionForRecentlyJoinedConsumer(consumer, readType)) : null; blockedByHash = lookAheadAllowed && readType == ReadType.Normal ? new MutableBoolean(false) : null; MutableInt permits = permitsForConsumer.computeIfAbsent(consumer, k -> new MutableInt(getAvailablePermits(consumer))); // a consumer was found for the sticky key hash and the entry can be dispatched - if (permits.intValue() > 0 && canDispatchEntry(entry, readType, stickyKeyHash, - maxLastSentPosition, blockedByHash)) { + if (permits.intValue() > 0 + && canDispatchEntry(consumer, entry, readType, stickyKeyHash, blockedByHash)) { // decrement the permits for the consumer permits.decrement(); // allow the entry to be dispatched @@ -491,6 +451,7 @@ private Map> filterAndGroupEntriesForDispatching(List> filterAndGroupEntriesForDispatching(List 0) { - return false; - } - // If redeliveryMessages contains messages that correspond to the same hash as the entry to be dispatched // do not send those messages for order guarantee if (readType == ReadType.Normal && redeliveryMessages.containsStickyKeyHash(stickyKeyHash)) { @@ -545,6 +501,16 @@ private boolean canDispatchEntry(Entry entry, return false; } + if (drainingHashesRequired) { + // If the hash is draining, do not send the message + if (drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash)) { + if (blockedByHash != null) { + blockedByHash.setTrue(); + } + return false; + } + } + return true; } @@ -566,8 +532,6 @@ private class ReplayPositionFilter implements Predicate { // tracks the available permits for each consumer for the duration of the filter usage // the filter is stateful and shouldn't be shared or reused later private final Map availablePermitsMap = new HashMap<>(); - private final Map maxLastSentPositionCache = - hasRecentlyJoinedConsumers() ? new HashMap<>() : null; @Override public boolean test(Position position) { @@ -585,6 +549,7 @@ public boolean test(Position position) { } return true; } + // find the consumer for the sticky key hash Consumer consumer = selector.select(stickyKeyHash.intValue()); // skip replaying the message position if there's no assigned consumer @@ -599,113 +564,32 @@ public boolean test(Position position) { if (availablePermits.intValue() <= 0) { return false; } - // check if the entry position can be replayed to a recently joined consumer - Position maxLastSentPosition = maxLastSentPositionCache != null - ? maxLastSentPositionCache.computeIfAbsent(consumer, __ -> - resolveMaxLastSentPositionForRecentlyJoinedConsumer(consumer, ReadType.Replay)) - : null; - if (maxLastSentPosition != null && position.compareTo(maxLastSentPosition) > 0) { + + if (drainingHashesRequired + && drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash.intValue())) { + // the hash is draining and the consumer is not the draining consumer return false; } + availablePermits.decrement(); return true; } } - /** - * Contains the logic to resolve the max last sent position for a consumer - * when the consumer has recently joined. This is only applicable for key shared mode when - * allowOutOfOrderDelivery=false. - */ - private Position resolveMaxLastSentPositionForRecentlyJoinedConsumer(Consumer consumer, ReadType readType) { - if (recentlyJoinedConsumers == null) { - return null; - } - removeConsumersFromRecentJoinedConsumers(); - Position maxLastSentPosition = recentlyJoinedConsumers.get(consumer); - // At this point, all the old messages were already consumed and this consumer - // is now ready to receive any message - if (maxLastSentPosition == null) { - // The consumer has not recently joined, so we can send all messages - return null; - } - - // If the read type is Replay, we should avoid send messages that hold by other consumer to the new consumers, - // For example, we have 10 messages [0,1,2,3,4,5,6,7,8,9] - // If the consumer0 get message 0 and 1, and does not acked message 0, then consumer1 joined, - // when consumer1 get message 2,3, the broker will not dispatch messages to consumer1 - // because of the mark delete position did not move forward. - // So message 2,3 will stored in the redeliver tracker. - // Now, consumer2 joined, it will read new messages from the cursor, - // so the recentJoinedPosition is 4 for consumer2 - // Because of there are messages need to redeliver, so the broker will read the redelivery message first [2,3] - // message [2,3] is lower than the recentJoinedPosition 4, - // so the message [2,3] will dispatched to the consumer2 - // But the message [2,3] should not dispatch to consumer2. - - if (readType == ReadType.Replay) { - Position minLastSentPositionForRecentJoinedConsumer = recentlyJoinedConsumers.values().iterator().next(); - if (minLastSentPositionForRecentJoinedConsumer != null - && minLastSentPositionForRecentJoinedConsumer.compareTo(maxLastSentPosition) < 0) { - maxLastSentPosition = minLastSentPositionForRecentJoinedConsumer; - } + @Override + protected int getStickyKeyHash(Entry entry) { + if (entry instanceof EntryAndMetadata entryAndMetadata) { + // use the cached sticky key hash if available, otherwise calculate the sticky key hash and cache it + return entryAndMetadata.getOrUpdateCachedStickyKeyHash(selector::makeStickyKeyHash); } - - return maxLastSentPosition; + return selector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer())); } - @Override public void markDeletePositionMoveForward() { - // Execute the notification in different thread to avoid a mutex chain here - // from the delete operation that was completed - topic.getBrokerService().getTopicOrderedExecutor().execute(() -> { - synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - if (hasRecentlyJoinedConsumers() - && removeConsumersFromRecentJoinedConsumers()) { - // After we process acks, we need to check whether the mark-delete position was advanced and we - // can finally read more messages. It's safe to call readMoreEntries() multiple times. - readMoreEntries(); - } - } - }); - } - - private boolean hasRecentlyJoinedConsumers() { - return !MapUtils.isEmpty(recentlyJoinedConsumers); - } - - private boolean removeConsumersFromRecentJoinedConsumers() { - if (MapUtils.isEmpty(recentlyJoinedConsumers)) { - return false; - } - Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); - boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; - Position mdp = cursor.getMarkDeletedPosition(); - if (mdp != null) { - while (itr.hasNext()) { - Map.Entry entry = itr.next(); - if (entry.getValue().compareTo(mdp) <= 0) { - itr.remove(); - hasConsumerRemovedFromTheRecentJoinedConsumers = true; - } else { - break; - } - } - } - return hasConsumerRemovedFromTheRecentJoinedConsumers; - } - - @Nullable - private synchronized Position updateIfNeededAndGetLastSentPosition() { - if (lastSentPosition == null) { - return null; - } - final Position mdp = cursor.getMarkDeletedPosition(); - if (mdp != null && mdp.compareTo(lastSentPosition) > 0) { - lastSentPosition = mdp; - } - return lastSentPosition; + // reschedule a read with a backoff after moving the mark-delete position forward since there might have + // been consumers that were blocked by hash and couldn't make progress + reScheduleReadWithKeySharedUnblockingInterval(); } /** @@ -829,34 +713,6 @@ public boolean hasSameKeySharedPolicy(KeySharedMeta ksm) { && ksm.isAllowOutOfOrderDelivery() == this.allowOutOfOrderDelivery); } - public LinkedHashMap getRecentlyJoinedConsumers() { - return recentlyJoinedConsumers; - } - - public synchronized String getLastSentPosition() { - if (lastSentPosition == null) { - return null; - } - return lastSentPosition.toString(); - } - - @VisibleForTesting - public Position getLastSentPositionField() { - return lastSentPosition; - } - - public synchronized String getIndividuallySentPositions() { - if (individuallySentPositions == null) { - return null; - } - return individuallySentPositions.toString(); - } - - @VisibleForTesting - public LongPairRangeSet getIndividuallySentPositionsField() { - return individuallySentPositions; - } - public Map> getConsumerKeyHashRanges() { return selector.getConsumerKeyHashRanges(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 9a0545e6f0ab2..b8d351bddf839 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -25,7 +25,6 @@ import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.Collections; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -1297,31 +1296,6 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge subStats.allowOutOfOrderDelivery = keySharedDispatcher.isAllowOutOfOrderDelivery(); subStats.keySharedMode = keySharedDispatcher.getKeySharedMode().toString(); - - LinkedHashMap recentlyJoinedConsumers = keySharedDispatcher - .getRecentlyJoinedConsumers(); - if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) { - recentlyJoinedConsumers.forEach((k, v) -> { - // The dispatcher allows same name consumers - final StringBuilder stringBuilder = new StringBuilder(); - stringBuilder.append("consumerName=").append(k.consumerName()) - .append(", consumerId=").append(k.consumerId()); - if (k.cnx() != null) { - stringBuilder.append(", address=").append(k.cnx().clientAddress()); - } - subStats.consumersAfterMarkDeletePosition.put(stringBuilder.toString(), v.toString()); - }); - } - final String lastSentPosition = ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher) - .getLastSentPosition(); - if (lastSentPosition != null) { - subStats.lastSentPosition = lastSentPosition; - } - final String individuallySentPositions = ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher) - .getIndividuallySentPositions(); - if (individuallySentPositions != null) { - subStats.individuallySentPositions = individuallySentPositions; - } } subStats.nonContiguousDeletedMessagesRanges = cursor.getTotalNonContiguousDeletedMessagesRange(); subStats.nonContiguousDeletedMessagesRangesSerializedSize = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java new file mode 100644 index 0000000000000..3554f29255227 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BooleanSupplier; +import java.util.function.LongSupplier; + +/** + * Reschedules reads so that the possible pending read is cancelled if it's waiting for more entries. + * This will prevent the dispatcher in getting blocked when there are entries in the replay queue + * that should be handled. This will also batch multiple calls together to reduce the number of + * operations. + */ +class RescheduleReadHandler { + private static final int UNSET = -1; + private static final int NO_PENDING_READ = 0; + private final AtomicLong maxReadOpCounter = new AtomicLong(UNSET); + private final LongSupplier readIntervalMsSupplier; + private final ScheduledExecutorService executor; + private final Runnable cancelPendingRead; + private final Runnable rescheduleReadImmediately; + private final BooleanSupplier hasPendingReadRequestThatMightWait; + private final LongSupplier readOpCounterSupplier; + private final BooleanSupplier hasEntriesInReplayQueue; + + RescheduleReadHandler(LongSupplier readIntervalMsSupplier, + ScheduledExecutorService executor, Runnable cancelPendingRead, + Runnable rescheduleReadImmediately, BooleanSupplier hasPendingReadRequestThatMightWait, + LongSupplier readOpCounterSupplier, + BooleanSupplier hasEntriesInReplayQueue) { + this.readIntervalMsSupplier = readIntervalMsSupplier; + this.executor = executor; + this.cancelPendingRead = cancelPendingRead; + this.rescheduleReadImmediately = rescheduleReadImmediately; + this.hasPendingReadRequestThatMightWait = hasPendingReadRequestThatMightWait; + this.readOpCounterSupplier = readOpCounterSupplier; + this.hasEntriesInReplayQueue = hasEntriesInReplayQueue; + } + + public void rescheduleRead() { + long readOpCountWhenPendingRead = + hasPendingReadRequestThatMightWait.getAsBoolean() ? readOpCounterSupplier.getAsLong() : NO_PENDING_READ; + if (maxReadOpCounter.compareAndSet(UNSET, readOpCountWhenPendingRead)) { + Runnable runnable = () -> { + // Read the current value of maxReadOpCounter and set it to UNSET, this will allow scheduling a next + // runnable + long maxReadOpCount = maxReadOpCounter.getAndSet(UNSET); + // Cancel a possible pending read if it's been waiting for more entries since the runnable was + // scheduled. This is detected by checking that the value of the readOpCounter has not changed + // since the runnable was scheduled. Canceling the read request will only be needed if there + // are entries in the replay queue. + if (maxReadOpCount != NO_PENDING_READ && readOpCounterSupplier.getAsLong() == maxReadOpCount + && hasEntriesInReplayQueue.getAsBoolean()) { + cancelPendingRead.run(); + } + // Re-schedule read immediately, or join the next scheduled read + rescheduleReadImmediately.run(); + }; + long rescheduleDelay = readIntervalMsSupplier.getAsLong(); + if (rescheduleDelay > 0) { + executor.schedule(runnable, rescheduleDelay, TimeUnit.MILLISECONDS); + } else { + runnable.run(); + } + } else { + // When there's a scheduled read, update the maxReadOpCounter to carry the state when the later scheduled + // read was done + long updatedValue = maxReadOpCounter.updateAndGet( + // Ignore updating if the value is UNSET + current -> current == UNSET ? UNSET : + // Prefer keeping NO_PENDING_READ if the latest value is NO_PENDING_READ + (readOpCountWhenPendingRead == NO_PENDING_READ ? NO_PENDING_READ : + // Otherwise, keep the maximum value + Math.max(current, readOpCountWhenPendingRead))); + // If the value was unset, it means that the runnable was already run and retrying is needed + // so that we don't miss any entries + if (updatedValue == UNSET) { + // Retry + rescheduleRead(); + } + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java index cc1eae475fa2d..7a4126fedec64 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java @@ -71,19 +71,30 @@ public boolean contains(long item1, long item2) { } } - public void removeUpTo(long item1, long item2) { + /** + * Remove all items up to (and including) the specified item. + * + * @param item1 the first part of the item key + * @param item2 the second part of the item key + * @return true if any bits were cleared + */ + public boolean removeUpTo(long item1, long item2) { + boolean bitsCleared = false; lock.writeLock().lock(); try { Map.Entry firstEntry = map.firstEntry(); while (firstEntry != null && firstEntry.getKey() <= item1) { if (firstEntry.getKey() < item1) { map.remove(firstEntry.getKey(), firstEntry.getValue()); + bitsCleared = true; } else { RoaringBitmap bitSet = firstEntry.getValue(); if (bitSet != null) { + bitsCleared |= bitSet.contains(0, item2); bitSet.remove(0, item2); if (bitSet.isEmpty()) { map.remove(firstEntry.getKey(), bitSet); + bitsCleared = true; } } break; @@ -93,6 +104,7 @@ public void removeUpTo(long item1, long item2) { } finally { lock.writeLock().unlock(); } + return bitsCleared; } public > Optional first(LongPairSet.LongPairFunction longPairConverter) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index 5641816ee0b80..ffcc3bf0881db 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectWriter; import java.io.IOException; @@ -234,4 +236,21 @@ public static void receiveMessagesInThreads(BiFunction, Message< }); }).toList()).join(); } + + private static long mockConsumerIdGenerator = 0; + + public static org.apache.pulsar.broker.service.Consumer createMockConsumer(String consumerName) { + long consumerId = mockConsumerIdGenerator++; + return createMockConsumer(consumerName, consumerName + " consumerId:" + consumerId, consumerId); + } + + public static org.apache.pulsar.broker.service.Consumer createMockConsumer(String consumerName, String toString, long consumerId) { + // without stubOnly, the mock will record method invocations and could run into OOME + org.apache.pulsar.broker.service.Consumer + consumer = mock(org.apache.pulsar.broker.service.Consumer.class, Mockito.withSettings().stubOnly()); + when(consumer.consumerName()).thenReturn(consumerName); + when(consumer.toString()).thenReturn(consumerName + " consumerId:" + consumerId); + when(consumer.consumerId()).thenReturn(consumerId); + return consumer; + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 26da4116d09cb..70c2b343ec584 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -18,8 +18,6 @@ */ package org.apache.pulsar.broker.admin; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -58,7 +56,6 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; import javax.ws.rs.client.InvocationCallback; import javax.ws.rs.client.WebTarget; import javax.ws.rs.core.Response.Status; @@ -68,8 +65,6 @@ import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedgerInfo; -import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -79,8 +74,6 @@ import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; -import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; -import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.broker.testcontext.SpyConfig; import org.apache.pulsar.client.admin.GetStatsOptions; @@ -128,7 +121,6 @@ import org.apache.pulsar.common.policies.data.BrokerInfo; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; @@ -145,10 +137,7 @@ import org.apache.pulsar.common.policies.data.TopicHashPositions; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.Codec; -import org.apache.pulsar.common.util.Murmur3_32Hash; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; import org.awaitility.Awaitility; @@ -3457,201 +3446,6 @@ public void testGetTtlDurationDefaultInSeconds() throws Exception { assertNull(seconds); } - @Test - public void testGetLastSentPositionWhenJoining() throws Exception { - final String topic = "persistent://prop-xyz/ns1/testGetLastSentPositionWhenJoining-" + UUID.randomUUID().toString(); - final String subName = "my-sub"; - @Cleanup - Producer producer = pulsarClient.newProducer() - .topic(topic) - .enableBatching(false) - .create(); - - @Cleanup - final Consumer consumer1 = pulsarClient.newConsumer() - .topic(topic) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .subscribe(); - - final int messages = 10; - MessageIdImpl messageId = null; - for (int i = 0; i < messages; i++) { - messageId = (MessageIdImpl) producer.send(("Hello Pulsar - " + i).getBytes()); - consumer1.receive(); - } - - @Cleanup - final Consumer consumer2 = pulsarClient.newConsumer() - .topic(topic) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .subscribe(); - - TopicStats stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - SubscriptionStats subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.getConsumers().size(), 2); - ConsumerStats consumerStats = subStats.getConsumers().stream() - .filter(s -> s.getConsumerName().equals(consumer2.getConsumerName())).findFirst().get(); - Assert.assertEquals(consumerStats.getLastSentPositionWhenJoining(), - PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()).toString()); - } - - @Test - public void testGetLastSentPosition() throws Exception { - final String topic = "persistent://prop-xyz/ns1/testGetLastSentPosition-" + UUID.randomUUID().toString(); - final String subName = "my-sub"; - @Cleanup - final Producer producer = pulsarClient.newProducer() - .topic(topic) - .enableBatching(false) - .create(); - final AtomicInteger counter = new AtomicInteger(); - @Cleanup - final Consumer consumer = pulsarClient.newConsumer() - .topic(topic) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .messageListener((c, msg) -> { - try { - c.acknowledge(msg); - counter.getAndIncrement(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }) - .subscribe(); - - TopicStats stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - SubscriptionStats subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertNull(subStats.getLastSentPosition()); - - final int messages = 10; - MessageIdImpl messageId = null; - for (int i = 0; i < messages; i++) { - messageId = (MessageIdImpl) producer.send(("Hello Pulsar - " + i).getBytes()); - } - - Awaitility.await().untilAsserted(() -> assertEquals(counter.get(), messages)); - - stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.getLastSentPosition(), PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()).toString()); - } - - @Test - public void testGetIndividuallySentPositions() throws Exception { - // The producer sends messages with two types of keys. - // The dispatcher sends keyA messages to consumer1. - // Consumer1 will not receive any messages. Its receiver queue size is 1. - // Consumer2 will receive and ack any messages immediately. - - final String topic = "persistent://prop-xyz/ns1/testGetIndividuallySentPositions-" + UUID.randomUUID().toString(); - final String subName = "my-sub"; - @Cleanup - final Producer producer = pulsarClient.newProducer() - .topic(topic) - .enableBatching(false) - .create(); - - final String consumer1Name = "c1"; - final String consumer2Name = "c2"; - - @Cleanup - final Consumer consumer1 = pulsarClient.newConsumer() - .topic(topic) - .consumerName(consumer1Name) - .receiverQueueSize(1) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .subscribe(); - - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get().get().getSubscription(subName).getDispatcher(); - final String keyA = "key-a"; - final String keyB = "key-b"; - final int hashA = Murmur3_32Hash.getInstance().makeHash(keyA.getBytes()); - - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - - // the selector returns consumer1 if keyA - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - - final String consumerName = hash == hashA ? consumer1Name : consumer2Name; - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - final AtomicInteger consumer2AckCounter = new AtomicInteger(); - @Cleanup - final Consumer consumer2 = pulsarClient.newConsumer() - .topic(topic) - .consumerName(consumer2Name) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .messageListener((c, msg) -> { - try { - c.acknowledge(msg); - consumer2AckCounter.getAndIncrement(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }) - .subscribe(); - - final LongPairRangeSet.LongPairConsumer positionRangeConverter = PositionFactory::create; - final LongPairRangeSet expectedIndividuallySentPositions = new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter); - - TopicStats stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - SubscriptionStats subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.getIndividuallySentPositions(), expectedIndividuallySentPositions.toString()); - - final Function sendFn = (key) -> { - try { - return (MessageIdImpl) producer.newMessage().key(key).value(("msg").getBytes()).send(); - } catch (PulsarClientException e) { - throw new RuntimeException(e); - } - }; - final List messageIdList = new ArrayList<>(); - - // the dispatcher can send keyA message, but then consumer1's receiver queue will be full - messageIdList.add(sendFn.apply(keyA)); - - // the dispatcher can send messages other than keyA - messageIdList.add(sendFn.apply(keyA)); - messageIdList.add(sendFn.apply(keyB)); - messageIdList.add(sendFn.apply(keyA)); - messageIdList.add(sendFn.apply(keyB)); - messageIdList.add(sendFn.apply(keyB)); - - assertEquals(messageIdList.size(), 6); - Awaitility.await().untilAsserted(() -> assertEquals(consumer2AckCounter.get(), 3)); - - // set expected value - expectedIndividuallySentPositions.addOpenClosed(messageIdList.get(1).getLedgerId(), messageIdList.get(1).getEntryId(), - messageIdList.get(2).getLedgerId(), messageIdList.get(2).getEntryId()); - expectedIndividuallySentPositions.addOpenClosed(messageIdList.get(3).getLedgerId(), messageIdList.get(3).getEntryId(), - messageIdList.get(5).getLedgerId(), messageIdList.get(5).getEntryId()); - - stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.getIndividuallySentPositions(), expectedIndividuallySentPositions.toString()); - } - @Test public void testPartitionedTopicMsgDelayedAggregated() throws Exception { final String topic = "persistent://prop-xyz/ns1/testPartitionedTopicMsgDelayedAggregated-" + UUID.randomUUID().toString(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java index 8b72411329c65..ea6ffa2d70dba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java @@ -126,10 +126,10 @@ public void testWithTimer(DelayedDeliveryTracker tracker, NavigableMap false); ((AbstractDelayedDeliveryTracker) tracker).run(timeout); - verify(dispatcher, times(1)).readMoreEntries(); + verify(dispatcher, times(1)).readMoreEntriesAsync(); // Add a message that has a delivery time just after the previous run. It will get delivered based on the // tick delay plus the last tick run. @@ -189,11 +189,11 @@ public void testAddMessageWithDeliverAtTimeAfterNowBeforeTickTimeFrequencyWithSt // Wait longer than the tick time plus the HashedWheelTimer's tick time to ensure that enough time has // passed where it would have been triggered if the tick time was doing the triggering. Thread.sleep(600); - verify(dispatcher, times(1)).readMoreEntries(); + verify(dispatcher, times(1)).readMoreEntriesAsync(); // Not wait for the message delivery to get triggered. Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> verify(dispatcher).readMoreEntries()); + .untilAsserted(() -> verify(dispatcher).readMoreEntriesAsync()); tracker.close(); } @@ -212,7 +212,7 @@ public void testAddMessageWithDeliverAtTimeAfterNowAfterTickTimeFrequencyWithStr // Wait long enough for the runnable to run, but not longer than the tick time. The point is that the delivery // should get scheduled early when the tick duration has passed since the last tick. Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> verify(dispatcher).readMoreEntries()); + .untilAsserted(() -> verify(dispatcher).readMoreEntriesAsync()); tracker.close(); } @@ -233,7 +233,7 @@ public void testAddMessageWithDeliverAtTimeAfterFullTickTimeWithStrict(DelayedDe // Not wait for the message delivery to get triggered. Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> verify(dispatcher).readMoreEntries()); + .untilAsserted(() -> verify(dispatcher).readMoreEntriesAsync()); tracker.close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java index 04aafc49b47e6..e2feb2050652b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java @@ -168,20 +168,20 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume Map> expectedResult = new HashMap<>(); assertThat(consumers.get(0).consumerName()).isEqualTo("consumer1"); expectedResult.put(consumers.get(0), Arrays.asList( - Range.of(95615213, 440020355), - Range.of(440020356, 455987436), - Range.of(1189794593, 1264144431))); + Range.of(14359, 18366), + Range.of(29991, 39817), + Range.of(52980, 60442))); assertThat(consumers.get(1).consumerName()).isEqualTo("consumer2"); expectedResult.put(consumers.get(1), Arrays.asList( - Range.of(939655188, 1189794592), - Range.of(1314727625, 1977451233), - Range.of(1977451234, 2016237253))); + Range.of(1, 6668), + Range.of(39818, 52979), + Range.of(60443, 63679), + Range.of(65184, 65535))); assertThat(consumers.get(2).consumerName()).isEqualTo("consumer3"); expectedResult.put(consumers.get(2), Arrays.asList( - Range.of(0, 95615212), - Range.of(455987437, 939655187), - Range.of(1264144432, 1314727624), - Range.of(2016237254, 2147483646))); + Range.of(6669, 14358), + Range.of(18367, 29990), + Range.of(63680, 65183))); Map> consumerKeyHashRanges = selector.getConsumerKeyHashRanges(); assertThat(consumerKeyHashRanges).containsExactlyInAnyOrderEntriesOf(expectedResult); @@ -195,7 +195,8 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume } previousRange = range; } - assertThat(allRanges.stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum()).isEqualTo(Integer.MAX_VALUE); + Range totalRange = selector.getKeyHashRange(); + assertThat(allRanges.stream().mapToInt(Range::size).sum()).isEqualTo(totalRange.size()); } @Test @@ -247,12 +248,12 @@ private static void printSelectionCountStats(Map consumerS private static void printConsumerRangesStats(ConsistentHashingStickyKeyConsumerSelector selector) { selector.getConsumerKeyHashRanges().entrySet().stream() .map(entry -> Map.entry(entry.getKey(), - entry.getValue().stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum())) + entry.getValue().stream().mapToInt(Range::size).sum())) .sorted(Map.Entry.comparingByKey(Comparator.comparing(Consumer::toString))) .forEach(entry -> System.out.println( String.format("consumer: %s total ranges size: %d ratio: %.2f%%", entry.getKey(), entry.getValue(), - ((double) entry.getValue() / (Integer.MAX_VALUE - 1)) * 100.0d))); + ((double) entry.getValue() / selector.getKeyHashRange().size()) * 100.0d))); } private static Consumer createMockConsumer(String consumerName, String toString, long id) { @@ -323,7 +324,7 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemoved() { selector.addConsumer(consumer); } - int hashRangeSize = Integer.MAX_VALUE; + int hashRangeSize = selector.getKeyHashRange().size(); int validationPointCount = 200; int increment = hashRangeSize / (validationPointCount + 1); List selectedConsumerBeforeRemoval = new ArrayList<>(); @@ -342,13 +343,14 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemoved() { for (Consumer removedConsumer : consumers) { selector.removeConsumer(removedConsumer); removedConsumers.add(removedConsumer); + Map> consumerKeyHashRanges = selector.getConsumerKeyHashRanges(); for (int i = 0; i < validationPointCount; i++) { int hash = i * increment; Consumer selected = selector.select(hash); Consumer expected = selectedConsumerBeforeRemoval.get(i); if (!removedConsumers.contains(expected)) { assertThat(selected.consumerId()).as("validationPoint %d, removed %s, hash %d ranges %s", i, - removedConsumer.toString(), hash, selector.getConsumerKeyHashRanges()).isEqualTo(expected.consumerId()); + removedConsumer.toString(), hash, consumerKeyHashRanges).isEqualTo(expected.consumerId()); } } } @@ -441,7 +443,7 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() { selector.addConsumer(consumer); } - int hashRangeSize = Integer.MAX_VALUE; + int hashRangeSize = selector.getKeyHashRange().size(); int validationPointCount = 200; int increment = hashRangeSize / (validationPointCount + 1); List selectedConsumerBeforeRemoval = new ArrayList<>(); @@ -473,10 +475,10 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() { } @Test - public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() { + public void testShouldContainMinimalMappingChangesWhenConsumerLeavesAndRejoins() { final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); final String consumerName = "consumer"; - final int numOfInitialConsumers = 25; + final int numOfInitialConsumers = 10; List consumers = new ArrayList<>(); for (int i = 0; i < numOfInitialConsumers; i++) { final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); @@ -484,6 +486,8 @@ public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() { selector.addConsumer(consumer); } + ConsumerHashAssignmentsSnapshot assignmentsBefore = selector.getConsumerHashAssignmentsSnapshot(); + Map> expected = selector.getConsumerKeyHashRanges(); assertThat(selector.getConsumerKeyHashRanges()).as("sanity check").containsExactlyInAnyOrderEntriesOf(expected); @@ -492,7 +496,15 @@ public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() { selector.addConsumer(consumers.get(0)); selector.addConsumer(consumers.get(numOfInitialConsumers / 2)); - assertThat(selector.getConsumerKeyHashRanges()).as("ranges shouldn't change").containsExactlyInAnyOrderEntriesOf(expected); + ConsumerHashAssignmentsSnapshot assignmentsAfter = selector.getConsumerHashAssignmentsSnapshot(); + int removedRangesSize = assignmentsBefore.diffRanges(assignmentsAfter).keySet().stream() + .mapToInt(Range::size) + .sum(); + double allowedremovedRangesPercentage = 1; // 1% + int hashRangeSize = selector.getKeyHashRange().size(); + int allowedremovedRanges = (int) (hashRangeSize * (allowedremovedRangesPercentage / 100.0d)); + assertThat(removedRangesSize).describedAs("Allow up to %d%% of total hash range size to be impacted", + allowedremovedRangesPercentage).isLessThan(allowedremovedRanges); } @Test @@ -501,7 +513,7 @@ public void testConsumersReconnect() { final String consumerName = "consumer"; final int numOfInitialConsumers = 50; final int validationPointCount = 200; - final List pointsToTest = pointsToTest(validationPointCount); + final List pointsToTest = pointsToTest(validationPointCount, selector.getKeyHashRange().size()); List consumers = new ArrayList<>(); for (int i = 0; i < numOfInitialConsumers; i++) { final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); @@ -537,13 +549,38 @@ public void testConsumersReconnect() { } } - private List pointsToTest(int validationPointCount) { + private List pointsToTest(int validationPointCount, int hashRangeSize) { List res = new ArrayList<>(); - int hashRangeSize = Integer.MAX_VALUE; final int increment = hashRangeSize / (validationPointCount + 1); for (int i = 0; i < validationPointCount; i++) { - res.add(i * increment); + res.add(Math.max(i * increment, hashRangeSize - 1)); } return res; } -} + + @Test(enabled = false) + public void testPerformanceOfAdding1000ConsumersWith100Points() { + // test that adding 1000 consumers with 100 points runs in a reasonable time. + // This takes about 1 second on Apple M3 + // this unit test can be used for basic profiling + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + for (int i = 0; i < 1000; i++) { + // use real class to avoid Mockito over head + final Consumer consumer = new Consumer("consumer" + i, 0) { + @Override + public int hashCode() { + return consumerName().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof Consumer) { + return consumerName().equals(((Consumer) obj).consumerName()); + } + return false; + } + }; + selector.addConsumer(consumer); + } + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java new file mode 100644 index 0000000000000..5c886b6eec9f3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.apache.pulsar.broker.BrokerTestUtil.createMockConsumer; +import static org.assertj.core.api.Assertions.assertThat; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.client.api.Range; +import org.testng.annotations.Test; + +@Test(groups = "broker-impl") +public class ConsumerHashAssignmentsSnapshotTest { + @Test + public void testMergeOverlappingRanges() { + SortedSet ranges = new TreeSet<>(); + ranges.add(Range.of(1, 5)); + ranges.add(Range.of(6, 10)); + ranges.add(Range.of(8, 12)); + ranges.add(Range.of(15, 20)); + ranges.add(Range.of(21, 25)); + + SortedSet expectedMergedRanges = new TreeSet<>(); + expectedMergedRanges.add(Range.of(1, 12)); + expectedMergedRanges.add(Range.of(15, 25)); + + List mergedRanges = ConsumerHashAssignmentsSnapshot.mergeOverlappingRanges(ranges); + + assertThat(mergedRanges).containsExactlyElementsOf(expectedMergedRanges); + } + + @Test + public void testDiffRanges_NoChanges() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).isEmpty(); + } + + @Test + public void testDiffRanges_ConsumerChanged() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer2)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).containsEntry(Range.of(1, 5), Pair.of(consumer1, consumer2)); + } + + @Test + public void testDiffRanges_RangeAdded() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + Consumer consumer1 = createMockConsumer("consumer1"); + + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).containsEntry(Range.of(1, 5), Pair.of(null, consumer1)); + } + + @Test + public void testDiffRanges_RangeRemoved() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).containsEntry(Range.of(1, 5), Pair.of(consumer1, null)); + } + + @Test + public void testDiffRanges_OverlappingRanges() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(3, 7), consumer2)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).containsEntry(Range.of(3, 5), Pair.of(consumer1, consumer2)); + } + + @Test + public void testResolveConsumerRemovedHashRanges_NoChanges() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).isEmpty(); + } + + @Test + public void testResolveConsumerRemovedHashRanges_ConsumerChanged() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer2)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).containsExactlyInAnyOrderEntriesOf( + Map.of(consumer1, RemovedHashRanges.of(List.of(Range.of(1, 5))))); + } + + @Test + public void testResolveConsumerRemovedHashRanges_RangeAdded() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).isEmpty(); + } + + @Test + public void testResolveConsumerRemovedHashRanges_RangeRemoved() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).containsExactlyInAnyOrderEntriesOf( + Map.of(consumer1, RemovedHashRanges.of(List.of(Range.of(1, 5))))); + } + + @Test + public void testResolveConsumerRemovedHashRanges_OverlappingRanges() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(3, 7), consumer2)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).containsExactlyInAnyOrderEntriesOf( + Map.of(consumer1, RemovedHashRanges.of(List.of(Range.of(3, 5))))); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java new file mode 100644 index 0000000000000..ecb20beeb648a --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.apache.pulsar.broker.BrokerTestUtil.createMockConsumer; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; +import org.apache.pulsar.broker.service.DrainingHashesTracker.UnblockingHandler; +import org.testng.annotations.Test; + +public class DrainingHashesTrackerTest { + @Test + public void addEntry_AddsNewEntry() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + + tracker.addEntry(consumer, 1); + + assertNotNull(tracker.getEntry(1)); + assertSame(tracker.getEntry(1).getConsumer(), consumer); + } + + @Test + public void addEntry_ThrowsExceptionForZeroStickyHash() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + + assertThrows(IllegalArgumentException.class, () -> tracker.addEntry(consumer, 0)); + } + + @Test + public void reduceRefCount_ReducesReferenceCount() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer, 1); + + tracker.reduceRefCount(consumer, 1, false); + + assertNull(tracker.getEntry(1)); + } + + @Test + public void reduceRefCount_DoesNotReduceForDifferentConsumer() { + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer1, 1); + + assertThrows(IllegalStateException.class, () -> tracker.reduceRefCount(consumer2, 1, false)); + + assertNotNull(tracker.getEntry(1)); + assertSame(tracker.getEntry(1).getConsumer(), consumer1); + } + + @Test + public void shouldBlockStickyKeyHash_DoesNotBlockForExistingEntryWhenSameConsumer() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer, 1); + + boolean result = tracker.shouldBlockStickyKeyHash(consumer, 1); + + assertFalse(result); + } + + @Test + public void shouldBlockStickyKeyHash_BlocksForExistingEntryWhenDifferentConsumer() { + Consumer consumer1 = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer1, 1); + + Consumer consumer2 = createMockConsumer("consumer2"); + boolean result = tracker.shouldBlockStickyKeyHash(consumer2, 1); + + assertTrue(result); + } + + + @Test + public void shouldBlockStickyKeyHash_DoesNotBlockForNewEntry() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + + boolean result = tracker.shouldBlockStickyKeyHash(consumer, 1); + + assertFalse(result); + } + + @Test + public void startBatch_IncrementsBatchLevel() { + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + + tracker.startBatch(); + assertEquals(tracker.batchLevel, 1); + + tracker.startBatch(); + assertEquals(tracker.batchLevel, 2); + + tracker.startBatch(); + assertEquals(tracker.batchLevel, 3); + } + + @Test + public void endBatch_DecrementsBatchLevel() { + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.startBatch(); + + tracker.endBatch(); + + assertEquals(tracker.batchLevel, 0); + } + + @Test + public void endBatch_InvokesUnblockingHandlerWhenUnblockedWhileBatching() { + // given a tracker with unblocking handler + UnblockingHandler unblockingHandler = mock(UnblockingHandler.class); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", unblockingHandler); + + // when a hash is draining + Consumer consumer1 = createMockConsumer("consumer1"); + tracker.addEntry(consumer1, 1); + // and batch starts + tracker.startBatch(); + + // when hash gets blocked + Consumer consumer2 = createMockConsumer("consumer2"); + tracker.shouldBlockStickyKeyHash(consumer2, 1); + // and it gets unblocked + tracker.reduceRefCount(consumer1, 1, false); + + // then no unblocking call should be done + verify(unblockingHandler, never()).stickyKeyHashUnblocked(anyInt()); + + // when batch ends + tracker.endBatch(); + // then unblocking call should be done + verify(unblockingHandler).stickyKeyHashUnblocked(-1); + } + + @Test + public void clear_RemovesAllEntries() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer, 1); + + tracker.clear(); + + assertNull(tracker.getEntry(1)); + } + + @Test + public void unblockingHandler_InvokesStickyKeyHashUnblocked() { + // given a tracker with unblocking handler + UnblockingHandler unblockingHandler = mock(UnblockingHandler.class); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", unblockingHandler); + + // when a hash is draining + Consumer consumer = createMockConsumer("consumer1"); + tracker.addEntry(consumer, 1); + // aand hash gets blocked + Consumer consumer2 = createMockConsumer("consumer2"); + tracker.shouldBlockStickyKeyHash(consumer2, 1); + // and hash gets unblocked + tracker.reduceRefCount(consumer, 1, false); + + // then unblocking call should be done + verify(unblockingHandler).stickyKeyHashUnblocked(1); + } + + @Test + public void unblockingHandler_DoesNotInvokeStickyKeyHashUnblockedWhenClosing() { + // given a tracker with unblocking handler + UnblockingHandler unblockingHandler = mock(UnblockingHandler.class); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", unblockingHandler); + + // when a hash is draining + Consumer consumer = createMockConsumer("consumer1"); + tracker.addEntry(consumer, 1); + // aand hash gets blocked + Consumer consumer2 = createMockConsumer("consumer2"); + tracker.shouldBlockStickyKeyHash(consumer2, 1); + // and hash gets unblocked + tracker.reduceRefCount(consumer, 1, true); + + // then unblocking call should be done + verify(unblockingHandler, never()).stickyKeyHashUnblocked(anyInt()); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java new file mode 100644 index 0000000000000..42f5935ca88ff --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.apache.pulsar.broker.BrokerTestUtil.createMockConsumer; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import java.util.ArrayList; +import java.util.List; +import org.testng.annotations.Test; + +public class PendingAcksMapTest { + @Test + public void addPendingAckIfAllowed_AddsAckWhenAllowed() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + + boolean result = pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + assertTrue(result); + assertTrue(pendingAcksMap.contains(1L, 1L)); + } + + @Test + public void addPendingAckIfAllowed_DoesNotAddAckWhenNotAllowed() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap.PendingAcksAddHandler addHandler = mock(PendingAcksMap.PendingAcksAddHandler.class); + when(addHandler.handleAdding(any(), anyLong(), anyLong(), anyInt())).thenReturn(false); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> addHandler, () -> null); + + boolean result = pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + assertFalse(result); + assertFalse(pendingAcksMap.contains(1L, 1L)); + } + + @Test + public void addPendingAckIfAllowed_DoesNotAddAfterClosed() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> {}); + + boolean result = pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + assertFalse(result); + assertFalse(pendingAcksMap.contains(1L, 1L)); + } + + @Test + public void forEach_ProcessesAllPendingAcks() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + + List processedEntries = new ArrayList<>(); + pendingAcksMap.forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> processedEntries.add(entryId)); + + assertEquals(processedEntries, List.of(1L, 2L)); + } + + @Test + public void forEachAndClose_ProcessesAndClearsAllPendingAcks() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + + List processedEntries = new ArrayList<>(); + pendingAcksMap.forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> processedEntries.add(entryId)); + + assertEquals(processedEntries, List.of(1L, 2L)); + assertEquals(pendingAcksMap.size(), 0); + } + + @Test + public void remove_RemovesPendingAck() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + boolean result = pendingAcksMap.remove(1L, 1L); + + assertTrue(result); + assertFalse(pendingAcksMap.contains(1L, 1L)); + } + + @Test + public void removeAllUpTo_RemovesAllPendingAcksUpToSpecifiedEntry() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125); + + pendingAcksMap.removeAllUpTo(1L, 2L); + + assertFalse(pendingAcksMap.contains(1L, 1L)); + assertFalse(pendingAcksMap.contains(1L, 2L)); + assertTrue(pendingAcksMap.contains(2L, 1L)); + } + + @Test + public void removeAllUpTo_RemovesAllPendingAcksUpToSpecifiedEntryAcrossMultipleLedgers() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125); + pendingAcksMap.addPendingAckIfAllowed(2L, 2L, 1, 126); + pendingAcksMap.addPendingAckIfAllowed(3L, 1L, 1, 127); + + pendingAcksMap.removeAllUpTo(2L, 1L); + + assertFalse(pendingAcksMap.contains(1L, 1L)); + assertFalse(pendingAcksMap.contains(1L, 2L)); + assertFalse(pendingAcksMap.contains(2L, 1L)); + assertTrue(pendingAcksMap.contains(2L, 2L)); + assertTrue(pendingAcksMap.contains(3L, 1L)); + } + + @Test + public void addPendingAckIfAllowed_InvokesAddHandler() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap.PendingAcksAddHandler addHandler = mock(PendingAcksMap.PendingAcksAddHandler.class); + when(addHandler.handleAdding(any(), anyLong(), anyLong(), anyInt())).thenReturn(true); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> addHandler, () -> null); + + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + verify(addHandler).handleAdding(consumer, 1L, 1L, 123); + } + + @Test + public void remove_InvokesRemoveHandler() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap.PendingAcksRemoveHandler removeHandler = mock(PendingAcksMap.PendingAcksRemoveHandler.class); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> removeHandler); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + pendingAcksMap.remove(1L, 1L); + + verify(removeHandler).handleRemoving(consumer, 1L, 1L, 123, false); + } + + @Test + public void removeAllUpTo_InvokesRemoveHandlerForEachEntry() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap.PendingAcksRemoveHandler removeHandler = mock(PendingAcksMap.PendingAcksRemoveHandler.class); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> removeHandler); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125); + + pendingAcksMap.removeAllUpTo(1L, 2L); + + verify(removeHandler).handleRemoving(consumer, 1L, 1L, 123, false); + verify(removeHandler).handleRemoving(consumer, 1L, 2L, 124, false); + verify(removeHandler, never()).handleRemoving(consumer, 2L, 1L, 125, false); + } + + @Test + public void size_ReturnsCorrectSize() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125); + + assertEquals(pendingAcksMap.size(), 3); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java index 2896c13af0093..4d79e7ccdf0d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -55,8 +54,6 @@ import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; -import org.apache.pulsar.broker.service.persistent.MessageRedeliveryController; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.schema.SchemaRegistry; @@ -1753,90 +1750,6 @@ public void testMessageRedelivery() throws Exception { producer.close(); } - /** - * Verify: 1. Broker should not replay already acknowledged messages 2. Dispatcher should not stuck while - * dispatching new messages due to previous-replay of invalid/already-acked messages - * - * @throws Exception - */ - @Test - public void testMessageReplay() throws Exception { - - final String topicName = "persistent://prop/ns-abc/topic2"; - final String subName = "sub2"; - - Message msg; - int totalMessages = 10; - int replayIndex = totalMessages / 2; - - Consumer consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subName) - .subscriptionType(SubscriptionType.Shared).receiverQueueSize(1).subscribe(); - Producer producer = pulsarClient.newProducer() - .topic(topicName) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); - - PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); - assertNotNull(topicRef); - PersistentSubscription subRef = topicRef.getSubscription(subName); - PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) subRef - .getDispatcher(); - Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class - .getDeclaredField("redeliveryMessages"); - redeliveryMessagesField.setAccessible(true); - MessageRedeliveryController redeliveryMessages = new MessageRedeliveryController(true); - - assertNotNull(subRef); - - // (1) Produce messages - for (int i = 0; i < totalMessages; i++) { - String message = "my-message-" + i; - producer.send(message.getBytes()); - } - - MessageIdImpl firstAckedMsg = null; - // (2) Consume and ack messages except first message - for (int i = 0; i < totalMessages; i++) { - msg = consumer.receive(); - consumer.acknowledge(msg); - MessageIdImpl msgId = (MessageIdImpl) msg.getMessageId(); - if (i == 0) { - firstAckedMsg = msgId; - } - if (i < replayIndex) { - // (3) accumulate acked messages for replay - redeliveryMessages.add(msgId.getLedgerId(), msgId.getEntryId()); - } - } - - // (4) redelivery : should redeliver only unacked messages - Thread.sleep(1000); - - redeliveryMessagesField.set(dispatcher, redeliveryMessages); - // (a) redelivery with all acked-message should clear messageReply bucket - dispatcher.redeliverUnacknowledgedMessages(dispatcher.getConsumers().get(0), DEFAULT_CONSUMER_EPOCH); - Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> { - return redeliveryMessages.isEmpty(); - }); - assertTrue(redeliveryMessages.isEmpty()); - - // (b) fill messageReplyBucket with already acked entry again: and try to publish new msg and read it - redeliveryMessages.add(firstAckedMsg.getLedgerId(), firstAckedMsg.getEntryId()); - redeliveryMessagesField.set(dispatcher, redeliveryMessages); - // send new message - final String testMsg = "testMsg"; - producer.send(testMsg.getBytes()); - // consumer should be able to receive only new message and not the - dispatcher.consumerFlow(dispatcher.getConsumers().get(0), 1); - msg = consumer.receive(1, TimeUnit.SECONDS); - assertNotNull(msg); - assertEquals(msg.getData(), testMsg.getBytes()); - - consumer.close(); - producer.close(); - } - @Test public void testCreateProducerWithSameName() throws Exception { String topic = "persistent://prop/ns-abc/testCreateProducerWithSameName"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java index a03ed92b81590..052c5ceb5cdde 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java @@ -158,8 +158,8 @@ public void testSkipReadEntriesFromCloseCursor() throws Exception { dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), null); return null; - }).when(cursor).asyncReadEntriesOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher), - Mockito.any(), Mockito.any()); + }).when(cursor).asyncReadEntriesWithSkipOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher), + Mockito.any(), Mockito.any(), Mockito.any()); dispatcher.readMoreEntries(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index a0054f7e71425..4b29ead984e7a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -37,8 +37,6 @@ 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.assertThrows; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.BoundType; @@ -51,16 +49,15 @@ import io.netty.util.concurrent.GenericFutureListener; import java.lang.reflect.Field; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Queue; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.mledger.Entry; @@ -73,6 +70,7 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryAndMetadata; import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; import org.apache.pulsar.broker.service.RedeliveryTracker; @@ -85,8 +83,6 @@ import org.apache.pulsar.common.policies.data.HierarchyTopicPolicies; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.awaitility.Awaitility; import org.mockito.ArgumentCaptor; import org.testng.Assert; @@ -123,9 +119,10 @@ public void setup() throws Exception { doReturn(true).when(configMock).isSubscriptionRedeliveryTrackerEnabled(); doReturn(100).when(configMock).getDispatcherMaxReadBatchSize(); doReturn(true).when(configMock).isSubscriptionKeySharedUseConsistentHashing(); - doReturn(1).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); + doReturn(20).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); doReturn(false).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); doReturn(false).when(configMock).isAllowOverrideEntryFilters(); + doReturn(false).when(configMock).isDispatchThrottlingOnNonBacklogConsumerEnabled(); doAnswer(invocation -> retryBackoffInitialTimeInMs).when(configMock).getDispatcherRetryBackoffInitialTimeInMs(); doAnswer(invocation -> retryBackoffMaxTimeInMs).when(configMock).getDispatcherRetryBackoffMaxTimeInMs(); pulsarMock = mock(PulsarService.class); @@ -223,6 +220,8 @@ protected static Consumer createMockConsumer() { TransportCnx transportCnx = mock(TransportCnx.class); doReturn(transportCnx).when(consumerMock).cnx(); doReturn(true).when(transportCnx).isActive(); + doReturn(100).when(consumerMock).getMaxUnackedMessages(); + doReturn(1).when(consumerMock).getAvgMessagesPerEntry(); return consumerMock; } @@ -321,13 +320,16 @@ public void testSendMessage() { @Test public void testSkipRedeliverTemporally() { final Consumer slowConsumerMock = createMockConsumer(); + AtomicInteger slowConsumerPermits = new AtomicInteger(0); + doAnswer(invocation -> slowConsumerPermits.get()).when(slowConsumerMock).getAvailablePermits(); + final ChannelPromise slowChannelMock = mock(ChannelPromise.class); // add entries to redeliver and read target final List redeliverEntries = new ArrayList<>(); - redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key1"))); + redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key123"))); final List readEntries = new ArrayList<>(); - readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); - readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key2"))); + readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key123"))); + readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key222"))); try { Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumers.class.getDeclaredField("totalAvailablePermits"); @@ -348,9 +350,6 @@ public void testSkipRedeliverTemporally() { // Create 2Consumers try { doReturn("consumer2").when(slowConsumerMock).consumerName(); - when(slowConsumerMock.getAvailablePermits()) - .thenReturn(0) - .thenReturn(1); doReturn(true).when(slowConsumerMock).isWritable(); doReturn(slowChannelMock).when(slowConsumerMock).sendMessages( anyList(), @@ -375,13 +374,12 @@ public void testSkipRedeliverTemporally() { // and then stop to dispatch to slowConsumer persistentDispatcher.readEntriesComplete(redeliverEntries, PersistentDispatcherMultipleConsumers.ReadType.Replay); - verify(consumerMock, times(1)).sendMessages( argThat(arg -> { assertEquals(arg.size(), 1); Entry entry = arg.get(0); assertEquals(entry.getLedgerId(), 1); - assertEquals(entry.getEntryId(), 3); + assertEquals(entry.getEntryId(), 1); return true; }), any(EntryBatchSizes.class), @@ -408,25 +406,9 @@ public void testMessageRedelivery() throws Exception { final Queue actualEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); final Queue expectedEntriesToConsumer1 = new ConcurrentLinkedQueue<>(); - expectedEntriesToConsumer1.add(PositionFactory.create(1, 1)); final Queue expectedEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); - expectedEntriesToConsumer2.add(PositionFactory.create(1, 2)); - expectedEntriesToConsumer2.add(PositionFactory.create(1, 3)); - - final AtomicInteger remainingEntriesNum = new AtomicInteger( - expectedEntriesToConsumer1.size() + expectedEntriesToConsumer2.size()); - - // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2 - final List allEntries = new ArrayList<>(); - allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key2"))); - allEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); - allEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key1"))); - allEntries.forEach(entry -> ((EntryImpl) entry).retain()); - final List redeliverEntries = new ArrayList<>(); - redeliverEntries.add(allEntries.get(0)); // message1 - final List readEntries = new ArrayList<>(); - readEntries.add(allEntries.get(2)); // message3 + final AtomicInteger remainingEntriesNum = new AtomicInteger(0); final Consumer consumer1 = createMockConsumer(); doReturn("consumer1").when(consumer1).consumerName(); @@ -434,8 +416,7 @@ public void testMessageRedelivery() throws Exception { when(consumer1.getAvailablePermits()).thenReturn(0).thenReturn(10); doReturn(true).when(consumer1).isWritable(); doAnswer(invocationOnMock -> { - @SuppressWarnings("unchecked") - List entries = (List) invocationOnMock.getArgument(0); + List entries = invocationOnMock.getArgument(0); for (Entry entry : entries) { remainingEntriesNum.decrementAndGet(); actualEntriesToConsumer1.add(entry.getPosition()); @@ -449,8 +430,7 @@ public void testMessageRedelivery() throws Exception { when(consumer2.getAvailablePermits()).thenReturn(10); doReturn(true).when(consumer2).isWritable(); doAnswer(invocationOnMock -> { - @SuppressWarnings("unchecked") - List entries = (List) invocationOnMock.getArgument(0); + List entries = invocationOnMock.getArgument(0); for (Entry entry : entries) { remainingEntriesNum.decrementAndGet(); actualEntriesToConsumer2.add(entry.getPosition()); @@ -467,44 +447,64 @@ public void testMessageRedelivery() throws Exception { totalAvailablePermitsField.setAccessible(true); totalAvailablePermitsField.set(persistentDispatcher, 1000); - final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class - .getDeclaredField("redeliveryMessages"); - redeliveryMessagesField.setAccessible(true); - MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField - .get(persistentDispatcher); - redeliveryMessages.add(allEntries.get(0).getLedgerId(), allEntries.get(0).getEntryId(), - getStickyKeyHash(allEntries.get(0))); // message1 - redeliveryMessages.add(allEntries.get(1).getLedgerId(), allEntries.get(1).getEntryId(), - getStickyKeyHash(allEntries.get(1))); // message2 + StickyKeyConsumerSelector selector = persistentDispatcher.getSelector(); + + String keyForConsumer1 = generateKeyForConsumer(selector, consumer1); + String keyForConsumer2 = generateKeyForConsumer(selector, consumer2); + + // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2 + final List allEntries = new ArrayList<>(); + allEntries.add(EntryAndMetadata.create(EntryImpl.create(1, 1, createMessage("message1", 1, keyForConsumer1)))); + allEntries.add(EntryAndMetadata.create(EntryImpl.create(1, 2, createMessage("message2", 2, keyForConsumer1)))); + allEntries.add(EntryAndMetadata.create(EntryImpl.create(1, 3, createMessage("message3", 3, keyForConsumer2)))); + allEntries.forEach(entry -> { + EntryImpl entryImpl = (EntryImpl) ((EntryAndMetadata) entry).unwrap(); + entryImpl.retain(); + // initialize sticky key hash + persistentDispatcher.getStickyKeyHash(entry); + }); + remainingEntriesNum.set(allEntries.size()); + + final List redeliverEntries = new ArrayList<>(); + redeliverEntries.add(allEntries.get(0)); // message1 + final List readEntries = new ArrayList<>(); + readEntries.add(allEntries.get(2)); // message3 + + expectedEntriesToConsumer1.add(allEntries.get(0).getPosition()); + expectedEntriesToConsumer1.add(allEntries.get(1).getPosition()); + expectedEntriesToConsumer2.add(allEntries.get(2).getPosition()); // Mock Cursor#asyncReplayEntries doAnswer(invocationOnMock -> { - @SuppressWarnings("unchecked") - Set positions = (Set) invocationOnMock.getArgument(0); - List entries = allEntries.stream().filter(entry -> positions.contains(entry.getPosition())) + Set positionsArg = invocationOnMock.getArgument(0); + Set positions = new TreeSet<>(positionsArg); + Set alreadyReceived = new TreeSet<>(); + alreadyReceived.addAll(actualEntriesToConsumer1); + alreadyReceived.addAll(actualEntriesToConsumer2); + List entries = allEntries.stream().filter(entry -> positions.contains(entry.getPosition()) + && !alreadyReceived.contains(entry.getPosition())) .collect(Collectors.toList()); - if (!entries.isEmpty()) { - ((PersistentStickyKeyDispatcherMultipleConsumers) invocationOnMock.getArgument(1)) - .readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay); - } - return Collections.emptySet(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = invocationOnMock.getArgument(1); + dispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay); + return alreadyReceived; }).when(cursorMock).asyncReplayEntries(anySet(), any(PersistentStickyKeyDispatcherMultipleConsumers.class), eq(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay), anyBoolean()); // Mock Cursor#asyncReadEntriesOrWait - AtomicBoolean asyncReadEntriesOrWaitCalled = new AtomicBoolean(); doAnswer(invocationOnMock -> { - if (asyncReadEntriesOrWaitCalled.compareAndSet(false, true)) { - ((PersistentStickyKeyDispatcherMultipleConsumers) invocationOnMock.getArgument(2)) - .readEntriesComplete(readEntries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal); - } else { - ((PersistentStickyKeyDispatcherMultipleConsumers) invocationOnMock.getArgument(2)) - .readEntriesComplete(Collections.emptyList(), PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal); - } + int maxEntries = invocationOnMock.getArgument(0); + Set alreadyReceived = new TreeSet<>(); + alreadyReceived.addAll(actualEntriesToConsumer1); + alreadyReceived.addAll(actualEntriesToConsumer2); + List entries = allEntries.stream() + .filter(entry -> !alreadyReceived.contains(entry.getPosition())) + .limit(maxEntries).collect(Collectors.toList()); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = invocationOnMock.getArgument(2); + dispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal); return null; - }).when(cursorMock).asyncReadEntriesOrWait(anyInt(), anyLong(), + }).when(cursorMock).asyncReadEntriesWithSkipOrWait(anyInt(), anyLong(), any(PersistentStickyKeyDispatcherMultipleConsumers.class), - eq(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal), any()); + eq(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal), any(), any()); // (1) Run sendMessagesToConsumers // (2) Attempts to send message1 to consumer1 but skipped because availablePermits is 0 @@ -512,6 +512,11 @@ public void testMessageRedelivery() throws Exception { // (4) Run readMoreEntries internally // (5) Run sendMessagesToConsumers internally // (6) Attempts to send message3 to consumer2 but skipped because redeliveryMessages contains message2 + redeliverEntries.forEach(entry -> { + EntryImpl entryImpl = (EntryImpl) ((EntryAndMetadata) entry).unwrap(); + entryImpl.retain(); + persistentDispatcher.addEntryToReplay(entry); + }); persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay, redeliverEntries, true); while (remainingEntriesNum.get() > 0) { @@ -525,313 +530,16 @@ public void testMessageRedelivery() throws Exception { allEntries.forEach(entry -> entry.release()); } - @DataProvider(name = "initializeLastSentPosition") - private Object[][] initialLastSentPositionProvider() { - return new Object[][] { { false }, { true } }; - } - - @Test(dataProvider = "initializeLastSentPosition") - public void testLastSentPositionAndIndividuallySentPositions(final boolean initializeLastSentPosition) throws Exception { - final Position initialLastSentPosition = PositionFactory.create(1, 10); - final LongPairRangeSet expectedIndividuallySentPositions - = new ConcurrentOpenLongPairRangeSet<>(4096, PositionFactory::create); - - final Field lastSentPositionField = PersistentStickyKeyDispatcherMultipleConsumers.class - .getDeclaredField("lastSentPosition"); - lastSentPositionField.setAccessible(true); - final LongPairRangeSet individuallySentPositions = persistentDispatcher.getIndividuallySentPositionsField(); - final Supplier clearPosition = () -> { - try { - lastSentPositionField.set(persistentDispatcher, initializeLastSentPosition ? initialLastSentPosition : null); - individuallySentPositions.clear(); - expectedIndividuallySentPositions.clear(); - } catch (Throwable e) { - return e; + private String generateKeyForConsumer(StickyKeyConsumerSelector selector, Consumer consumer) { + int i = 0; + while (!Thread.currentThread().isInterrupted()) { + String key = "key" + i++; + Consumer selectedConsumer = selector.select(key.getBytes(UTF_8)); + if (selectedConsumer == consumer) { + return key; } - return null; - }; - if (!initializeLastSentPosition) { - doReturn(initialLastSentPosition).when(cursorMock).getMarkDeletedPosition(); - doAnswer(invocationOnMock -> { - // skip copy operation - return initialLastSentPosition; - }).when(cursorMock).processIndividuallyDeletedMessagesAndGetMarkDeletedPosition(any()); } - - // Assume the range sequence is [1:0, 1:19], [2:0, 2:19], ..., [10:0, 10:19] - doAnswer((invocationOnMock -> { - final Position position = invocationOnMock.getArgument(0); - if (position.getEntryId() > 0) { - return PositionFactory.create(position.getLedgerId(), position.getEntryId() - 1); - } else if (position.getLedgerId() > 0) { - return PositionFactory.create(position.getLedgerId() - 1, 19); - } else { - throw new NullPointerException(); - } - })).when(ledgerMock).getPreviousPosition(any(Position.class)); - doAnswer((invocationOnMock -> { - final Position position = invocationOnMock.getArgument(0); - if (position.getEntryId() < 19) { - return PositionFactory.create(position.getLedgerId(), position.getEntryId() + 1); - } else { - return PositionFactory.create(position.getLedgerId() + 1, 0); - } - })).when(ledgerMock).getNextValidPosition(any(Position.class)); - doReturn(PositionFactory.create(10, 19)).when(ledgerMock).getLastConfirmedEntry(); - doAnswer((invocationOnMock -> { - final Range range = invocationOnMock.getArgument(0); - Position fromPosition = range.lowerEndpoint(); - boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; - Position toPosition = range.upperEndpoint(); - boolean toIncluded = range.upperBoundType() == BoundType.CLOSED; - - if (fromPosition.getLedgerId() == toPosition.getLedgerId()) { - // If the 2 positions are in the same ledger - long count = toPosition.getEntryId() - fromPosition.getEntryId() - 1; - count += fromIncluded ? 1 : 0; - count += toIncluded ? 1 : 0; - return count; - } else { - long count = 0; - // If the from & to are pointing to different ledgers, then we need to : - // 1. Add the entries in the ledger pointed by toPosition - count += toPosition.getEntryId(); - count += toIncluded ? 1 : 0; - - // 2. Add the entries in the ledger pointed by fromPosition - count += 20 - (fromPosition.getEntryId() + 1); - count += fromIncluded ? 1 : 0; - - // 3. Add the whole ledgers entries in between - for (long i = fromPosition.getLedgerId() + 1; i < toPosition.getLedgerId(); i++) { - count += 20; - } - - return count; - } - })).when(ledgerMock).getNumberOfEntries(any()); - assertEquals(ledgerMock.getNextValidPosition(PositionFactory.create(1, 0)), PositionFactory.create(1, 1)); - assertEquals(ledgerMock.getNextValidPosition(PositionFactory.create(1, 19)), PositionFactory.create(2, 0)); - assertEquals(ledgerMock.getPreviousPosition(PositionFactory.create(2, 0)), PositionFactory.create(1, 19)); - assertThrows(NullPointerException.class, () -> ledgerMock.getPreviousPosition(PositionFactory.create(0, 0))); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, 0), PositionFactory.create(1, 0))), 0); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, -1), PositionFactory.create(1, 9))), 10); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, 19), PositionFactory.create(2, -1))), 0); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, 19), PositionFactory.create(2, 9))), 10); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, -1), PositionFactory.create(3, 19))), 60); - - // Add a consumer - final Consumer consumer1 = createMockConsumer(); - doReturn("consumer1").when(consumer1).consumerName(); - when(consumer1.getAvailablePermits()).thenReturn(1000); - doReturn(true).when(consumer1).isWritable(); - doReturn(channelMock).when(consumer1).sendMessages(anyList(), any(EntryBatchSizes.class), - any(EntryBatchIndexesAcks.class), anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); - persistentDispatcher.addConsumer(consumer1); - - /* - On single ledger - */ - - // Expected individuallySentPositions (isp): [(1:-1, 1:8]] (init) -> [(1:-1, 1:9]] (update) -> [] (remove) - // Expected lastSentPosition (lsp): 1:10 (init) -> 1:10 (remove) - // upper bound and the new entry are less than initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 8); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 9, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:-1, 1:9]] -> [(1:-1, 1:10]] -> [] - // lsp: 1:10 -> 1:10 - // upper bound is less than initial last sent position - // upper bound and the new entry are less than or equal to initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 9); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 10, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:-1, 1:2], (1:3, 1:4], (1:5, 1:6]] -> [(1:-1, 1:2], (1:3, 1:4], (1:5, 1:6], (1:9, 1:10]] -> [] - // lsp: 1:10 -> 1:10 - // upper bound and the new entry are less than or equal to initial last sent position - // individually sent positions has multiple ranges - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 2); - individuallySentPositions.addOpenClosed(1, 3, 1, 4); - individuallySentPositions.addOpenClosed(1, 5, 1, 6); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 10, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:-1, 1:10]] -> [(1:-1, 1:11]] -> [] - // lsp: 1:10 -> 1:11 - // upper bound is less than or equal to initial last sent position - // the new entry is next position of initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 10); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 11).toString()); - - // isp: [(1:-1, 1:9]] -> [(1:-1, 1:9], (1:10, 1:11]] -> [] - // lsp: 1:10 -> 1:11 - // upper bound is less than initial last sent position - // the new entry is next position of initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 9); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 11).toString()); - - // isp: [(1:11, 1:15]] -> [(1:10, 1:15]] -> [] - // lsp: 1:10 -> 1:15 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry is next position of initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 15).toString()); - - // isp: [(1:11, 1:15]] -> [(1:10, 1:16]] -> [] - // lsp: 1:10 -> 1:16 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entries contain next position of initial last sent position - // first of the new entries is less than initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 9, createMessage("test", 1)), - EntryImpl.create(1, 11, createMessage("test", 2)), - EntryImpl.create(1, 16, createMessage("test", 3))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 16).toString()); - - // isp: [(1:11, 1:15]] -> [(1:11, 1:15]] -> [(1:11, 1:15]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the range contains the new entry - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 15); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 15, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:11, 1:15]] -> [(1:11, 1:16]] -> [(1:11, 1:16]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the range doesn't contain the new entry - // the new entry is next position of upper bound - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 16); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 16, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:11, 1:15]] -> [(1:11, 1:15], (1:16, 1:17]] -> [(1:11, 1:15], (1:16, 1:17]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the range doesn't contain the new entry - // the new entry isn't next position of upper bound - // the new entry is same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 15); - expectedIndividuallySentPositions.addOpenClosed(1, 16, 1, 17); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 17, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - /* - On multiple contiguous ledgers - */ - - // isp: [(1:11, 1:18]] -> [(1:11, 1:18], (2:-1, 2:0]] -> [(1:11, 1:18], (2:-1, 2:0]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the range doesn't contain the new entry - // the new entry isn't next position of upper bound - // the new entry isn't same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 18); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 18); - expectedIndividuallySentPositions.addOpenClosed(2, -1, 2, 0); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(2, 0, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:11, 1:19], (2:-1, 2:0]] -> [(1:10, 1:19], (2:-1, 2:0]] -> [] - // lsp: 1:10 -> 2:0 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry is next position of initial last sent position - // the new entry isn't same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 19); - individuallySentPositions.addOpenClosed(2, -1, 2, 0); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(2, 0).toString()); - - // isp: [(1:11, 1:19], (2:-1, 2:19], (3:-1, 3:0]] -> [(1:10, 1:19], (2:-1, 2:19], (3:-1, 3:0]] -> [] - // lsp: 1:10 -> 3:0 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry is next position of initial last sent position - // the new entry isn't same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 19); - individuallySentPositions.addOpenClosed(2, -1, 2, 19); - individuallySentPositions.addOpenClosed(3, -1, 3, 0); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(3, 0).toString()); - - // isp: [(1:11, 1:19], (2:-1, 2:0]] -> [(1:11, 1:19], (2:-1, 2:1]] -> [(1:11, 1:19], (2:-1, 2:1]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the new entry isn't same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 19); - individuallySentPositions.addOpenClosed(2, -1, 2, 0); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 19); - expectedIndividuallySentPositions.addOpenClosed(2, -1, 2, 1); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(2, 1, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + return null; } @DataProvider(name = "testBackoffDelayWhenNoMessagesDispatched") @@ -872,7 +580,7 @@ protected void reScheduleReadInMs(long readAfterMs) { // call "readEntriesComplete" directly to test the retry behavior List entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 1); assertEquals(retryDelays.get(0), 10, "Initial retry delay should be 10ms"); @@ -880,7 +588,7 @@ protected void reScheduleReadInMs(long readAfterMs) { ); // test the second retry delay entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 2); double delay = retryDelays.get(1); @@ -890,7 +598,7 @@ protected void reScheduleReadInMs(long readAfterMs) { // verify the max retry delay for (int i = 0; i < 100; i++) { entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); } Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 102); @@ -901,14 +609,14 @@ protected void reScheduleReadInMs(long readAfterMs) { // unblock to check that the retry delay is reset consumerMockAvailablePermits.set(1000); entries = List.of(EntryImpl.create(1, 2, createMessage("message2", 1, "key2"))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); // wait that the possibly async handling has completed Awaitility.await().untilAsserted(() -> assertFalse(dispatcher.isSendInProgress())); // now block again to check the next retry delay so verify it was reset consumerMockAvailablePermits.set(0); entries = List.of(EntryImpl.create(1, 3, createMessage("message3", 1, "key3"))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 103); assertEquals(retryDelays.get(0), 10, "Resetted retry delay should be 10ms"); @@ -955,7 +663,7 @@ protected void reScheduleReadInMs(long readAfterMs) { // call "readEntriesComplete" directly to test the retry behavior List entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 1); assertEquals(retryDelays.get(0), 0, "Initial retry delay should be 0ms"); @@ -963,7 +671,7 @@ protected void reScheduleReadInMs(long readAfterMs) { ); // test the second retry delay entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 2); double delay = retryDelays.get(1); @@ -973,7 +681,7 @@ protected void reScheduleReadInMs(long readAfterMs) { // verify the max retry delay for (int i = 0; i < 100; i++) { entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); } Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 102); @@ -984,14 +692,14 @@ protected void reScheduleReadInMs(long readAfterMs) { // unblock to check that the retry delay is reset consumerMockAvailablePermits.set(1000); entries = List.of(EntryImpl.create(1, 2, createMessage("message2", 1, "key2"))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); // wait that the possibly async handling has completed Awaitility.await().untilAsserted(() -> assertFalse(dispatcher.isSendInProgress())); // now block again to check the next retry delay so verify it was reset consumerMockAvailablePermits.set(0); entries = List.of(EntryImpl.create(1, 3, createMessage("message3", 1, "key3"))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 103); assertEquals(retryDelays.get(0), 0, "Resetted retry delay should be 0ms"); @@ -1092,9 +800,4 @@ private ByteBuf createMessage(String message, int sequenceId, String key) { .setPublishTime(System.currentTimeMillis()); return serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, Unpooled.copiedBuffer(message.getBytes(UTF_8))); } - - private int getStickyKeyHash(Entry entry) { - byte[] stickyKey = Commands.peekStickyKey(entry.getDataBuffer(), topicName, subscriptionName); - return StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey); - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java new file mode 100644 index 0000000000000..cf404e38b66d3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; +import java.util.function.LongSupplier; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class RescheduleReadHandlerTest { + private LongSupplier readIntervalMsSupplier; + private ScheduledExecutorService executor; + private Runnable cancelPendingRead; + private Runnable rescheduleReadImmediately; + private BooleanSupplier hasPendingReadRequestThatMightWait; + private LongSupplier readOpCounterSupplier; + private BooleanSupplier hasEntriesInReplayQueue; + private RescheduleReadHandler rescheduleReadHandler; + + @BeforeMethod + public void setUp() { + readIntervalMsSupplier = mock(LongSupplier.class); + executor = mock(ScheduledExecutorService.class); + cancelPendingRead = mock(Runnable.class); + rescheduleReadImmediately = mock(Runnable.class); + hasPendingReadRequestThatMightWait = mock(BooleanSupplier.class); + readOpCounterSupplier = mock(LongSupplier.class); + hasEntriesInReplayQueue = mock(BooleanSupplier.class); + rescheduleReadHandler = new RescheduleReadHandler(readIntervalMsSupplier, executor, cancelPendingRead, + () -> rescheduleReadImmediately.run(), hasPendingReadRequestThatMightWait, readOpCounterSupplier, hasEntriesInReplayQueue); + } + + @Test + public void rescheduleReadImmediately() { + when(readIntervalMsSupplier.getAsLong()).thenReturn(0L); + + rescheduleReadHandler.rescheduleRead(); + + verify(rescheduleReadImmediately).run(); + verify(executor, never()).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + } + + @Test + public void rescheduleReadWithDelay() { + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + + rescheduleReadHandler.rescheduleRead(); + + verify(rescheduleReadImmediately, never()).run(); + verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS)); + } + + @Test + public void rescheduleReadWithDelayAndCancelPendingRead() { + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + when(hasPendingReadRequestThatMightWait.getAsBoolean()).thenReturn(true); + when(readOpCounterSupplier.getAsLong()).thenReturn(5L); + when(hasEntriesInReplayQueue.getAsBoolean()).thenReturn(true); + doAnswer(invocation -> { + Runnable runnable = invocation.getArgument(0); + runnable.run(); + return null; + }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + + rescheduleReadHandler.rescheduleRead(); + + verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS)); + verify(rescheduleReadImmediately).run(); + verify(cancelPendingRead).run(); + } + + @Test + public void rescheduleReadWithDelayAndDontCancelPendingReadIfNoEntriesInReplayQueue() { + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + when(hasPendingReadRequestThatMightWait.getAsBoolean()).thenReturn(true); + when(readOpCounterSupplier.getAsLong()).thenReturn(5L); + when(hasEntriesInReplayQueue.getAsBoolean()).thenReturn(false); + doAnswer(invocation -> { + Runnable runnable = invocation.getArgument(0); + runnable.run(); + return null; + }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + + rescheduleReadHandler.rescheduleRead(); + + verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS)); + verify(rescheduleReadImmediately).run(); + verify(cancelPendingRead, never()).run(); + } + + @Test + public void rescheduleReadBatching() { + when(readOpCounterSupplier.getAsLong()).thenReturn(5L); + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + AtomicReference scheduledRunnable = new AtomicReference<>(); + doAnswer(invocation -> { + Runnable runnable = invocation.getArgument(0); + if (!scheduledRunnable.compareAndSet(null, runnable)) { + runnable.run(); + } + return null; + }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + // 3 rescheduleRead calls + rescheduleReadHandler.rescheduleRead(); + rescheduleReadHandler.rescheduleRead(); + rescheduleReadHandler.rescheduleRead(); + // scheduled task runs + scheduledRunnable.get().run(); + // verify that rescheduleReadImmediately is called only once + verify(rescheduleReadImmediately, times(1)).run(); + } + + @Test + public void rescheduleReadWithoutCancelPendingReadWhenReadOpCounterIncrements() { + // given + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + when(hasPendingReadRequestThatMightWait.getAsBoolean()).thenReturn(true); + when(readOpCounterSupplier.getAsLong()).thenReturn(5L).thenReturn(6L); + when(hasEntriesInReplayQueue.getAsBoolean()).thenReturn(true); + doAnswer(invocation -> { + Runnable runnable = invocation.getArgument(0); + runnable.run(); + return null; + }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + // when rescheduleRead is called + rescheduleReadHandler.rescheduleRead(); + // then verify calls + verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS)); + verify(rescheduleReadImmediately).run(); + // verify that cancelPendingRead is not called + verify(cancelPendingRead, never()).run(); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java index bc4cb73e5b6fe..4a8e7077395ed 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java @@ -38,12 +38,10 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicName; @@ -83,48 +81,6 @@ protected void cleanup() throws Exception { super.internalCleanup(); } - @Test - public void testConsumersAfterMarkDelete() throws PulsarClientException, PulsarAdminException { - final String topicName = "persistent://my-property/my-ns/testConsumersAfterMarkDelete-" - + UUID.randomUUID(); - final String subName = "my-sub"; - - Consumer consumer1 = pulsarClient.newConsumer() - .topic(topicName) - .receiverQueueSize(10) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .subscribe(); - - Producer producer = pulsarClient.newProducer() - .topic(topicName) - .create(); - - final int messages = 100; - for (int i = 0; i < messages; i++) { - producer.send(String.valueOf(i).getBytes()); - } - - // Receive by do not ack the message, so that the next consumer can added to the recentJoinedConsumer of the dispatcher. - consumer1.receive(); - - Consumer consumer2 = pulsarClient.newConsumer() - .topic(topicName) - .receiverQueueSize(10) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .subscribe(); - - TopicStats stats = admin.topics().getStats(topicName); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue() - .getConsumersAfterMarkDeletePosition().size(), 1); - - consumer1.close(); - consumer2.close(); - producer.close(); - } - @Test public void testNonContiguousDeletedMessagesRanges() throws Exception { final String topicName = "persistent://my-property/my-ns/testNonContiguousDeletedMessagesRanges-" diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index c08c37b413f4f..2b16647f5590c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -19,11 +19,9 @@ package org.apache.pulsar.client.api; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; import static org.apache.pulsar.broker.BrokerTestUtil.receiveMessages; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -32,7 +30,6 @@ import static org.testng.Assert.fail; import com.google.common.collect.Lists; import java.io.IOException; -import java.lang.reflect.Field; import java.nio.file.Files; import java.nio.file.Paths; import java.time.Duration; @@ -41,7 +38,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -51,6 +47,7 @@ import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -59,33 +56,30 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; +import java.util.function.BiFunction; import java.util.stream.Collectors; import lombok.Cleanup; +import lombok.SneakyThrows; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.DrainingHashesTracker; +import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; -import org.apache.pulsar.broker.service.persistent.MessageRedeliveryController; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.impl.ConsumerImpl; -import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.api.proto.KeySharedMode; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.schema.KeyValue; -import org.apache.pulsar.common.util.Murmur3_32Hash; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; +import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.mockito.Mockito; import org.slf4j.Logger; @@ -102,6 +96,7 @@ public class KeySharedSubscriptionTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(KeySharedSubscriptionTest.class); private static final List keys = Arrays.asList("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"); + private static final String SUBSCRIPTION_NAME = "key_shared"; @DataProvider(name = "batch") public Object[] batchProvider() { @@ -169,7 +164,9 @@ public void resetDefaultNamespace() throws Exception { defaultConf.getKeySharedLookAheadMsgInReplayThresholdPerConsumer()); } - private static final Random random = new Random(System.nanoTime()); + // Use a fixed seed to make the tests using random values deterministic + // When a test fails, it's possible to re-run it to reproduce the issue + private static final Random random = new Random(1); private static final int NUMBER_OF_KEYS = 300; @Test(dataProvider = "data") @@ -260,6 +257,8 @@ public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(bo Consumer consumer3 = createConsumer(topic, KeySharedPolicy.stickyHashRange() .ranges(Range.of(40001, KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE-1))); + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + @Cleanup Producer producer = createProducer(topic, enableBatch); @@ -269,11 +268,10 @@ public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(bo for (int i = 0; i < 10; i++) { for (String key : keys) { - int slot = Murmur3_32Hash.getInstance().makeHash(key.getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; - if (slot <= 20000) { + int stickyKeyHash = selector.makeStickyKeyHash(key.getBytes()); + if (stickyKeyHash <= 20000) { consumer1ExpectMessages++; - } else if (slot <= 40000) { + } else if (stickyKeyHash <= 40000) { consumer2ExpectMessages++; } else { consumer3ExpectMessages++; @@ -383,6 +381,8 @@ public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelect Consumer consumer3 = createConsumer(topic, KeySharedPolicy.stickyHashRange() .ranges(Range.of(40001, KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE-1))); + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + @Cleanup Producer producer = createProducer(topic, enableBatch); @@ -396,11 +396,10 @@ public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelect .send(); String fallbackKey = producer.getProducerName() + "-" + producer.getLastSequenceId(); - int slot = Murmur3_32Hash.getInstance().makeHash(fallbackKey.getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; - if (slot <= 20000) { + int stickyKeyHash = selector.makeStickyKeyHash(fallbackKey.getBytes()); + if (stickyKeyHash <= 20000) { consumer1ExpectMessages++; - } else if (slot <= 40000) { + } else if (stickyKeyHash <= 40000) { consumer2ExpectMessages++; } else { consumer3ExpectMessages++; @@ -460,6 +459,8 @@ public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(boole Consumer consumer3 = createConsumer(topic, KeySharedPolicy.stickyHashRange() .ranges(Range.of(40001, KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE-1))); + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + @Cleanup Producer producer = createProducer(topic, enableBatch); @@ -469,11 +470,10 @@ public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(boole for (int i = 0; i < 10; i++) { for (String key : keys) { - int slot = Murmur3_32Hash.getInstance().makeHash(key.getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; - if (slot <= 20000) { + int stickyKeyHash = selector.makeStickyKeyHash(key.getBytes()); + if (stickyKeyHash <= 20000) { consumer1ExpectMessages++; - } else if (slot <= 40000) { + } else if (stickyKeyHash <= 40000) { consumer2ExpectMessages++; } else { consumer3ExpectMessages++; @@ -502,7 +502,7 @@ public void testDisableKeySharedSubscription() throws PulsarClientException { @Cleanup Consumer c = pulsarClient.newConsumer() .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .ackTimeout(10, TimeUnit.SECONDS) .subscribe(); @@ -540,7 +540,6 @@ public void testCannotUseAcknowledgeCumulative() throws PulsarClientException { @Test(dataProvider = "batch") public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exception { String topic = "testMakingProgressWithSlowerConsumer-" + UUID.randomUUID(); - String slowKey = "slowKey"; List clients = new ArrayList<>(); @@ -556,16 +555,15 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc Consumer c = client.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) - .receiverQueueSize(1) + .receiverQueueSize(100) .messageListener((consumer, msg) -> { try { if (slowKey.equals(msg.getKey())) { // Block the thread to simulate a slow consumer Thread.sleep(10000); } - receivedMessages.incrementAndGet(); consumer.acknowledge(msg); } catch (Exception e) { @@ -576,6 +574,11 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc consumers.add(c); } + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + + org.apache.pulsar.broker.service.Consumer slowConsumer = + selector.select(selector.makeStickyKeyHash(slowKey.getBytes())); + @Cleanup Producer producer = createProducer(topic, enableBatch); @@ -587,18 +590,24 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc int N = 1000; + int nonSlowMessages = 0; + // Then send all the other keys for (int i = 0; i < N; i++) { + String key = String.valueOf(random.nextInt(NUMBER_OF_KEYS)); + if (selector.select(selector.makeStickyKeyHash(key.getBytes())) != slowConsumer) { + // count messages that are not going to the slow consumer + nonSlowMessages++; + } producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(key) .value(i) .send(); } - // Since only 1 out of 10 consumers is stuck, we should be able to receive ~90% messages, - // plus or minus for some skew in the key distribution. + int finalNonSlowMessages = nonSlowMessages; Awaitility.await().untilAsserted(() -> { - assertEquals((double) receivedMessages.get(), N * 0.9, N * 0.3); + assertThat(receivedMessages.get()).isGreaterThanOrEqualTo(finalNonSlowMessages); }); for (Consumer c : consumers) { @@ -614,6 +623,7 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc @Test public void testOrderingWhenAddingConsumers() throws Exception { String topic = "testOrderingWhenAddingConsumers-" + UUID.randomUUID(); + int numberOfKeys = 10; @Cleanup Producer producer = createProducer(topic, false); @@ -623,12 +633,14 @@ public void testOrderingWhenAddingConsumers() throws Exception { for (int i = 0; i < 10; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } - // All the already published messages will be pre-fetched by C1. + PendingAcksMap c1PendingAcks = getDispatcher(topic, SUBSCRIPTION_NAME).getConsumers().get(0).getPendingAcks(); + // Wait until all the already published messages have been pre-fetched by C1. + Awaitility.await().ignoreExceptions().until(() -> c1PendingAcks.size() == 10); // Adding a new consumer. @Cleanup @@ -636,11 +648,14 @@ public void testOrderingWhenAddingConsumers() throws Exception { for (int i = 10; i < 20; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } + Message message = c2.receive(100, TimeUnit.MILLISECONDS); + assertThat(message).describedAs("All keys should be blocked by ").isNull(); + // Closing c1, would trigger all messages to go to c2 c1.close(); @@ -652,6 +667,12 @@ public void testOrderingWhenAddingConsumers() throws Exception { } } + @SneakyThrows + private PersistentStickyKeyDispatcherMultipleConsumers getDispatcher(String topic, String subscription) { + return (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get() + .get().getSubscription(subscription).getDispatcher(); + } + @Test public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { String topic = "testReadAheadWithConfiguredLookAheadLimit-" + UUID.randomUUID(); @@ -665,7 +686,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { @Cleanup Consumer c1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .subscribe(); @@ -683,7 +704,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { @Cleanup Consumer c2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .subscribe(); @@ -701,7 +722,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { Thread.sleep(1000); Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); - PersistentSubscription sub = (PersistentSubscription) t.getSubscription("key_shared"); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(SUBSCRIPTION_NAME); // We need to ensure that dispatcher does not keep to look ahead in the topic, Position readPosition = sub.getCursor().getReadPosition(); @@ -712,6 +733,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { @Test public void testRemoveFirstConsumer() throws Exception { String topic = "testReadAheadWhenAddingConsumers-" + UUID.randomUUID(); + int numberOfKeys = 10; @Cleanup Producer producer = createProducer(topic, false); @@ -719,7 +741,7 @@ public void testRemoveFirstConsumer() throws Exception { @Cleanup Consumer c1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .consumerName("c1") @@ -727,7 +749,7 @@ public void testRemoveFirstConsumer() throws Exception { for (int i = 0; i < 10; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } @@ -740,7 +762,7 @@ public void testRemoveFirstConsumer() throws Exception { @Cleanup Consumer c2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .consumerName("c2") @@ -748,13 +770,13 @@ public void testRemoveFirstConsumer() throws Exception { for (int i = 10; i < 20; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } // C2 will not be able to receive any messages until C1 is done processing whatever he got prefetched - assertNull(c2.receive(100, TimeUnit.MILLISECONDS)); + assertNull(c2.receive(1, TimeUnit.SECONDS)); c1.close(); @@ -777,8 +799,7 @@ public void testHashRangeConflict() throws PulsarClientException { Consumer consumer2 = createFixedHashRangesConsumer(topic, sub, Range.of(100,399)); Assert.assertTrue(consumer2.isConnected()); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) pulsar - .getBrokerService().getTopicReference(topic).get().getSubscription(sub).getDispatcher(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, sub); Assert.assertEquals(dispatcher.getConsumers().size(), 2); try { @@ -887,6 +908,7 @@ public void testAttachKeyToMessageMetadata() throws PulsarClientException { public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { int defaultTTLSec = 3; int totalMessages = 1000; + int numberOfKeys = 50; this.conf.setTtlDurationDefaultInSeconds(defaultTTLSec); final String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); final String subName = "my-sub"; @@ -894,11 +916,15 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { @Cleanup Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) + .consumerName("consumer1") .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subName); + StickyKeyConsumerSelector selector = dispatcher.getSelector(); + @Cleanup Producer producer = pulsarClient.newProducer(Schema.INT32) .topic(topic) @@ -906,20 +932,26 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { for (int i = 0; i < totalMessages; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } - // don't ack the first message - consumer1.receive(); - consumer1.acknowledge(consumer1.receive()); + Set blockedHashes = new HashSet<>(); + // pull up to numberOfKeys messages and don't ack them + for (int i = 0; i < numberOfKeys + 1; i++) { + Message received = consumer1.receive(); + int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); + log.info("Received message {} with sticky key hash: {}", received.getMessageId(), stickyKeyHash); + blockedHashes.add(stickyKeyHash); + } - // The consumer1 and consumer2 should be stuck because of the mark delete position did not move forward. + // The consumer1 and consumer2 should be stuck since all hashes are blocked @Cleanup Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) + .consumerName("consumer2") .subscriptionName(subName) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); @@ -929,11 +961,19 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { received = consumer2.receive(1, TimeUnit.SECONDS); } catch (PulsarClientException ignore) { } - Assert.assertNull(received); + if (received != null) { + int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); + DrainingHashesTracker.DrainingHashEntry entry = + dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + + "included in blockedHashes=%s", + received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); + } @Cleanup Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) + .consumerName("consumer3") .subscriptionName(subName) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); @@ -942,7 +982,14 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { received = consumer3.receive(1, TimeUnit.SECONDS); } catch (PulsarClientException ignore) { } - Assert.assertNull(received); + if (received != null) { + int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); + DrainingHashesTracker.DrainingHashEntry entry = + dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + + "included in blockedHashes=%s", + received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); + } Optional topicRef = pulsar.getBrokerService().getTopic(topic, false).get(); assertTrue(topicRef.isPresent()); @@ -952,14 +999,23 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { // The mark delete position is move forward, so the consumers should receive new messages now. for (int i = 0; i < totalMessages; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } - // Wait broker dispatch messages. - Assert.assertNotNull(consumer2.receive(1, TimeUnit.SECONDS)); - Assert.assertNotNull(consumer3.receive(1, TimeUnit.SECONDS)); + Map receivedMessagesCountByConsumer = new ConcurrentHashMap<>(); + receiveMessages((consumer, message) -> { + consumer.acknowledgeAsync(message); + receivedMessagesCountByConsumer.computeIfAbsent(consumer.getConsumerName(), id -> new AtomicInteger(0)) + .incrementAndGet(); + return true; + }, Duration.ofSeconds(2), consumer1, consumer2, consumer3); + + assertThat(receivedMessagesCountByConsumer.values().stream().mapToInt(AtomicInteger::intValue) + .sum()).isGreaterThanOrEqualTo(totalMessages); + assertThat(receivedMessagesCountByConsumer.values()).allSatisfy( + count -> assertThat(count.get()).isGreaterThan(0)); } @Test(dataProvider = "partitioned") @@ -1151,15 +1207,8 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr producer.send("message".getBytes()); Awaitility.await().untilAsserted(() -> assertNotNull(consumer1.receive(100, TimeUnit.MILLISECONDS))); - CompletableFuture> future = pulsar.getBrokerService().getTopicIfExists(topicName); - assertTrue(future.isDone()); - assertTrue(future.get().isPresent()); - Topic topic = future.get().get(); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subName).getDispatcher(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topicName, subName); assertTrue(dispatcher.isAllowOutOfOrderDelivery()); - assertNull(dispatcher.getLastSentPositionField()); - assertNull(dispatcher.getIndividuallySentPositionsField()); consumer1.close(); final Consumer consumer2 = pulsarClient.newConsumer() @@ -1171,14 +1220,8 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr producer.send("message".getBytes()); Awaitility.await().untilAsserted(() -> assertNotNull(consumer2.receive(100, TimeUnit.MILLISECONDS))); - future = pulsar.getBrokerService().getTopicIfExists(topicName); - assertTrue(future.isDone()); - assertTrue(future.get().isPresent()); - topic = future.get().get(); - dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subName).getDispatcher(); + dispatcher = getDispatcher(topicName, subName); assertFalse(dispatcher.isAllowOutOfOrderDelivery()); - assertNotNull(dispatcher.getLastSentPositionField()); - assertNotNull(dispatcher.getIndividuallySentPositionsField()); consumer2.close(); } @@ -1250,7 +1293,7 @@ public void testCheckConsumersWithSameName() throws Exception { } }); - l.await(); + l.await(10, TimeUnit.SECONDS); } @DataProvider(name = "preSend") @@ -1258,366 +1301,6 @@ private Object[][] preSendProvider() { return new Object[][] { { false }, { true } }; } - @Test(timeOut = 30_000, dataProvider = "preSend") - public void testCheckBetweenSkippingAndRecentlyJoinedConsumers(boolean preSend) throws Exception { - conf.setSubscriptionKeySharedUseConsistentHashing(true); - - final String topicName = "persistent://public/default/recently-joined-consumers-" + UUID.randomUUID(); - final String subName = "my-sub"; - - @Cleanup - final Producer p = pulsarClient.newProducer(Schema.STRING) - .topic(topicName) - .create(); - if (preSend) { - // verify that the test succeeds even if the topic has a message - p.send("msg"); - } - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.STRING) - .topic(topicName) - .subscriptionInitialPosition(SubscriptionInitialPosition.Latest) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)); - - // create 2 consumers - final String c1ConsumerName = "c1"; - @Cleanup - final Consumer c1 = cb.get().consumerName(c1ConsumerName).receiverQueueSize(1).subscribe(); - @Cleanup - final Consumer c2 = cb.get().consumerName("c2").receiverQueueSize(1000).subscribe(); - - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - final Field recentlyJoinedConsumersField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("recentlyJoinedConsumers"); - recentlyJoinedConsumersField.setAccessible(true); - final LinkedHashMap recentlyJoinedConsumers = (LinkedHashMap) recentlyJoinedConsumersField.get(dispatcher); - final String keyA = "key-a"; - final int hashA = Murmur3_32Hash.getInstance().makeHash(keyA.getBytes()); - final Map hashConsumerMap = new HashMap<>(); - hashConsumerMap.put(hashA, c1.getConsumerName()); - - // enforce the selector will return c1 if keyA - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - // send and receive - Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getConsumers().stream().filter(c -> c.getConsumerName().equals(c1ConsumerName)).findFirst().get().getAvailablePermits(), 1)); - final MessageIdImpl msg0Id = (MessageIdImpl) p.newMessage().key(keyA).value("msg-0").send(); - Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getConsumers().stream().filter(c -> c.getConsumerName().equals(c1ConsumerName)).findFirst().get().getAvailablePermits(), 0)); - - final MessageIdImpl msg1Id = (MessageIdImpl) p.newMessage().key(keyA).value("msg-1").send(); - Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getMsgBacklog(), 2)); - - final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class - .getDeclaredField("redeliveryMessages"); - redeliveryMessagesField.setAccessible(true); - final MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField.get(dispatcher); - - final Set replayMsgSet = redeliveryMessages.getMessagesToReplayNow(3, item -> true); - assertEquals(replayMsgSet.size(), 1); - final Position replayMsg = replayMsgSet.stream().findAny().get(); - assertEquals(replayMsg, PositionFactory.create(msg1Id.getLedgerId(), msg1Id.getEntryId())); - - // add c3 - final String c3ConsumerName = "c3"; - hashConsumerMap.put(hashA, c3ConsumerName); - @Cleanup - final Consumer c3 = cb.get().consumerName(c3ConsumerName).subscribe(); - final List> c3Msgs = new ArrayList<>(); - final org.apache.pulsar.broker.service.Consumer c3Broker = dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(c3ConsumerName)).findFirst().get(); - assertEquals(recentlyJoinedConsumers.get(c3Broker), PositionFactory.create(msg0Id.getLedgerId(), msg0Id.getEntryId())); - - // None of messages are sent to c3. - Message c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); - assertNull(c3Msg); - - // Disconnect c1 - c1.close(); - - c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); - assertNotNull(c3Msg); - c3Msgs.add(c3Msg); - // The mark delete position will move forward. Then remove c3 from recentlyJoinedConsumers. - c3.acknowledge(c3Msg); - Awaitility.await().untilAsserted(() -> assertNull(recentlyJoinedConsumers.get(c3Broker))); - c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); - assertNotNull(c3Msg); - c3Msgs.add(c3Msg); - c3.acknowledge(c3Msg); - - // check ordering - assertTrue(c3Msgs.get(0).getMessageId().compareTo(c3Msgs.get(1).getMessageId()) < 0); - } - - @Test(timeOut = 30_000) - public void testLastSentPositionWhenRecreatingDispatcher() throws Exception { - // The lastSentPosition and individuallySentPositions should be initialized - // by the markDeletedPosition and individuallyDeletedMessages. - final String topicName = "persistent://public/default/rewind-" + UUID.randomUUID(); - final String subName = "my-sub"; - - final int numMessages = 9; - final List keys = Arrays.asList("key-a", "key-b", "key-c"); - final AtomicInteger receiveCounter = new AtomicInteger(); - final AtomicInteger ackCounter = new AtomicInteger(); - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.INT32) - .topic(topicName) - .enableBatching(false) - .create(); - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)); - - @Cleanup - final Consumer c1 = cb.get().messageListener((c, msg) -> { - if (keys.get(0).equals(msg.getKey())) { - try { - c.acknowledge(msg); - ackCounter.getAndIncrement(); - } catch (PulsarClientException e) { - fail(e.getMessage()); - } - } - receiveCounter.getAndIncrement(); - }).subscribe(); - - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - final ManagedCursorImpl cursor = (ManagedCursorImpl) ((PersistentSubscription) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName)).getCursor(); - final ManagedLedgerImpl ledger = (ManagedLedgerImpl) cursor.getManagedLedger(); - - MessageIdImpl msgId = null; - for (int i = 0; i < numMessages; i++) { - msgId = (MessageIdImpl) producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); - } - - // wait for consumption - Awaitility.await().untilAsserted(() -> assertEquals(receiveCounter.get(), numMessages)); - assertEquals(ackCounter.get(), numMessages / keys.size()); - assertEquals(dispatcher.getLastSentPositionField(), PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); - assertTrue(individuallySentPositionsField.isEmpty()); - receiveCounter.set(0); - ackCounter.set(0); - - // create expected values - final Position expectedLastSentPosition = ledger.getNextValidPosition(cursor.getMarkDeletedPosition()); - final ConcurrentOpenLongPairRangeSet - expectedIndividuallySentPositions = new ConcurrentOpenLongPairRangeSet<>(4096, PositionFactory::create); - cursor.getIndividuallyDeletedMessagesSet().forEach(range -> { - final Position lower = range.lowerEndpoint(); - final Position upper = range.upperEndpoint(); - expectedIndividuallySentPositions.addOpenClosed(lower.getLedgerId(), lower.getEntryId(), upper.getLedgerId(), upper.getEntryId()); - return true; - }); - - // modify subscription type to close current dispatcher - admin.topics().createSubscription(topicName, "sub-alt", MessageId.earliest); - c1.close(); - @Cleanup - final Consumer c2 = pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Exclusive) - .subscribe(); - c2.close(); - assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getType(), SubscriptionType.Exclusive.toString()); - - @Cleanup - final Consumer c3 = cb.get().receiverQueueSize(0).subscribe(); - dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - - assertNull(dispatcher.getLastSentPositionField()); - assertTrue(individuallySentPositionsField.isEmpty()); - - assertNotNull(c3.receive()); - - // validate the individuallySentPosition is initialized by the individuallyDeletedMessages - // if it is not initialized expectedly, it has sent-hole of key-c messages because key-c messages are not scheduled to be dispatched to some consumer(already acked). - assertEquals(dispatcher.getLastSentPositionField(), expectedLastSentPosition); - assertEquals(individuallySentPositionsField.toString(), expectedIndividuallySentPositions.toString()); - } - - @Test(timeOut = 30_000) - public void testLastSentPositionWhenResettingCursor() throws Exception { - // The lastSentPosition and individuallySentPositions should be cleared if reset-cursor operation is executed. - final String nsName = "public/default"; - final String topicName = "persistent://" + nsName + "/reset-cursor-" + UUID.randomUUID(); - final String subName = "my-sub"; - - final int numMessages = 10; - final List keys = Arrays.asList("key-a", "key-b"); - final AtomicInteger ackCounter = new AtomicInteger(); - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.INT32) - .topic(topicName) - .enableBatching(false) - .create(); - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .receiverQueueSize(0) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)); - - @Cleanup - final Consumer c1 = cb.get().consumerName("c1").subscribe(); - @Cleanup - final Consumer c2 = cb.get().consumerName("c2").subscribe(); - - // set retention policy - admin.namespaces().setRetention(nsName, new RetentionPolicies(1, 1024 * 1024)); - - // enforce the selector will return c1 if keys.get(0) - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - final int hashA = Murmur3_32Hash.getInstance().makeHash(keys.get(0).getBytes()); - final Map hashConsumerMap = new HashMap<>(); - hashConsumerMap.put(hashA, c1.getConsumerName()); - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - for (int i = 0; i < numMessages; i++) { - producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); - } - - // consume some messages - for (int i = 0; i < numMessages / keys.size(); i++) { - final Message msg = c2.receive(); - if (msg != null) { - c2.acknowledge(msg); - ackCounter.getAndIncrement(); - } - } - assertEquals(ackCounter.get(), numMessages / keys.size()); - - // store current lastSentPosition for comparison - final LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - assertNotNull(dispatcher.getLastSentPositionField()); - assertFalse(individuallySentPositionsField.isEmpty()); - - // reset cursor and receive a message - admin.topics().resetCursor(topicName, subName, MessageId.earliest, true); - - // validate the lastSentPosition and individuallySentPositions are cleared after resetting cursor - assertNull(dispatcher.getLastSentPositionField()); - assertTrue(individuallySentPositionsField.isEmpty()); - } - - @Test(timeOut = 30_000) - public void testLastSentPositionWhenSkipping() throws Exception { - // The lastSentPosition and individuallySentPositions should be updated if skip operation is executed. - // There are updated to follow the new markDeletedPosition. - final String topicName = "persistent://public/default/skip-" + UUID.randomUUID(); - final String subName = "my-sub"; - - final int numMessages = 10; - final List keys = Arrays.asList("key-a", "key-b"); - final int numSkip = 2; - final AtomicInteger ackCounter = new AtomicInteger(); - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.INT32) - .topic(topicName) - .enableBatching(false) - .create(); - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)) - .receiverQueueSize(0); - - @Cleanup - final Consumer c1 = cb.get().consumerName("c1").subscribe(); - @Cleanup - final Consumer c2 = cb.get().consumerName("c2").subscribe(); - - // enforce the selector will return c1 if keys.get(0) - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - final int hashA = Murmur3_32Hash.getInstance().makeHash(keys.get(0).getBytes()); - final Map hashConsumerMap = new HashMap<>(); - hashConsumerMap.put(hashA, c1.getConsumerName()); - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - final List positionList = new ArrayList<>(); - for (int i = 0; i < numMessages; i++) { - final MessageIdImpl msgId = (MessageIdImpl) producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); - positionList.add(PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); - } - - // consume some messages - for (int i = 0; i < numSkip; i++) { - final Message msg = c2.receive(); - if (msg != null) { - c2.acknowledge(msg); - ackCounter.getAndIncrement(); - } - } - assertEquals(ackCounter.get(), numSkip); - final ManagedCursorImpl managedCursor = ((ManagedCursorImpl) ((PersistentSubscription) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName)).getCursor()); - Awaitility.await().untilAsserted(() -> assertEquals(managedCursor.getIndividuallyDeletedMessagesSet().size(), 2)); - - // store current lastSentPosition for comparison - final Position lastSentPositionBeforeSkip = dispatcher.getLastSentPositionField(); - final LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - assertNotNull(lastSentPositionBeforeSkip); - assertFalse(individuallySentPositionsField.isEmpty()); - - // skip messages and receive a message - admin.topics().skipMessages(topicName, subName, numSkip); - final MessageIdImpl msgIdAfterSkip = (MessageIdImpl) c1.receive().getMessageId(); - final Position positionAfterSkip = PositionFactory.create(msgIdAfterSkip.getLedgerId(), - msgIdAfterSkip.getEntryId()); - assertEquals(positionAfterSkip, positionList.get(4)); - - // validate the lastSentPosition is updated to the new markDeletedPosition - // validate the individuallySentPositions is updated expectedly (removeAtMost the new markDeletedPosition) - final Position lastSentPosition = dispatcher.getLastSentPositionField(); - assertNotNull(lastSentPosition); - assertTrue(lastSentPosition.compareTo(lastSentPositionBeforeSkip) > 0); - assertEquals(lastSentPosition, positionList.get(4)); - assertTrue(individuallySentPositionsField.isEmpty()); - } - private KeySharedMode getKeySharedModeOfSubscription(Topic topic, String subscription) { if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.persistent)) { return ((PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subscription) @@ -1665,7 +1348,7 @@ private Consumer createConsumer(String topic, KeySharedPolicy keyShared throws PulsarClientException { ConsumerBuilder builder = pulsarClient.newConsumer(Schema.INT32); builder.topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .ackTimeout(3, TimeUnit.SECONDS); if (keySharedPolicy != null) { @@ -1927,8 +1610,8 @@ public void testStickyKeyRangesRestartConsumers() throws Exception { }}); // wait for some messages to be received by both of the consumers - count1.await(); - count2.await(); + count1.await(5, TimeUnit.SECONDS); + count2.await(5, TimeUnit.SECONDS); consumer1.close(); consumer2.close(); @@ -1974,7 +1657,7 @@ public void testStickyKeyRangesRestartConsumers() throws Exception { }) .subscribe(); // wait for all the messages to be delivered - count3.await(); + count3.await(20, TimeUnit.SECONDS); assertTrue(sentMessages.isEmpty(), "didn't receive " + sentMessages); producerFuture.get(); @@ -2085,7 +1768,8 @@ private AtomicInteger injectReplayReadCounter(String topicName, String cursorNam @Test public void testNoRepeatedReadAndDiscard() throws Exception { int delayedMessages = 100; - final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + int numberOfKeys = delayedMessages; + final String topic = newUniqueName("persistent://public/default/tp"); final String subName = "my-sub"; admin.topics().createNonPartitionedTopic(topic); AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); @@ -2095,7 +1779,7 @@ public void testNoRepeatedReadAndDiscard() throws Exception { Producer producer = pulsarClient.newProducer(Schema.INT32).topic(topic).enableBatching(false).create(); for (int i = 0; i < delayedMessages; i++) { MessageId messageId = producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(random.nextInt(numberOfKeys))) .value(100 + i) .send(); log.info("Published message :{}", messageId); @@ -2103,12 +1787,14 @@ public void testNoRepeatedReadAndDiscard() throws Exception { producer.close(); // Make ack holes. + @Cleanup Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); + @Cleanup Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) @@ -2136,7 +1822,7 @@ public void testNoRepeatedReadAndDiscard() throws Exception { redeliverConsumer = consumer1; } - // consumer3 will be added to the "recentJoinedConsumers". + @Cleanup Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) @@ -2145,17 +1831,10 @@ public void testNoRepeatedReadAndDiscard() throws Exception { .subscribe(); redeliverConsumer.close(); + Thread.sleep(5000); // Verify: no repeated Read-and-discard. - Thread.sleep(5 * 1000); int maxReplayCount = delayedMessages * 2; - log.info("Reply read count: {}", replyReadCounter.get()); - assertTrue(replyReadCounter.get() < maxReplayCount); - - // cleanup. - consumer1.close(); - consumer2.close(); - consumer3.close(); - admin.topics().delete(topic, false); + assertThat(replyReadCounter.get()).isLessThanOrEqualTo(maxReplayCount); } @DataProvider(name = "allowKeySharedOutOfOrder") @@ -2184,7 +1863,7 @@ public Object[][] allowKeySharedOutOfOrder() { public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedOutOfOrder) throws Exception { final int messagesSentPerTime = 100; final Set totalReceivedMessages = new TreeSet<>(); - final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topic = newUniqueName("persistent://public/default/tp"); final String subName = "my-sub"; admin.topics().createNonPartitionedTopic(topic); AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); @@ -2243,7 +1922,7 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO msgList2.add(msg2); } Message msg3 = consumer3.receive(1, TimeUnit.SECONDS); - if (msg2 != null) { + if (msg3 != null) { totalReceivedMessages.add(msg3.getValue()); msgList3.add(msg3); } @@ -2251,23 +1930,35 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO Consumer consumerWillBeClose = null; Consumer consumerAlwaysAck = null; Consumer consumerStuck = null; + Runnable consumerStuckAckHandler; + if (!msgList1.isEmpty()) { msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); consumerAlwaysAck = consumer1; consumerWillBeClose = consumer2; consumerStuck = consumer3; + consumerStuckAckHandler = () -> { + msgList3.forEach(msg -> consumer3.acknowledgeAsync(msg)); + }; } else if (!msgList2.isEmpty()){ msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); consumerAlwaysAck = consumer2; consumerWillBeClose = consumer3; consumerStuck = consumer1; + consumerStuckAckHandler = () -> { + msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); + }; } else { msgList3.forEach(msg -> consumer3.acknowledgeAsync(msg)); consumerAlwaysAck = consumer3; consumerWillBeClose = consumer1; consumerStuck = consumer2; + consumerStuckAckHandler = () -> { + msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); + }; } + // 2. Add consumer4 after "consumerWillBeClose" was close, and consumer4 will be stuck due to the mechanism // "recentlyJoinedConsumers". Consumer consumer4 = pulsarClient.newConsumer(Schema.INT32) @@ -2314,6 +2005,7 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO log.info("Reply read count: {}", replyReadCounter.get()); assertTrue(replyReadCounter.get() < maxReplayCount); // Verify: at last, all messages will be received. + consumerStuckAckHandler.run(); ReceivedMessages receivedMessages = ackAllMessages(consumerAlwaysAck, consumerStuck, consumer4); totalReceivedMessages.addAll(receivedMessages.messagesReceived.stream().map(p -> p.getRight()).collect( Collectors.toList())); @@ -2340,7 +2032,7 @@ public void testReadAheadLimit() throws Exception { Producer producer = createProducer(topic, false); // create a consumer and close it to create a subscription - String subscriptionName = "key_shared"; + String subscriptionName = SUBSCRIPTION_NAME; pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subscriptionName) @@ -2348,11 +2040,7 @@ public void testReadAheadLimit() throws Exception { .subscribe() .close(); - Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); - PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscriptionName); - // get the dispatcher reference - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subscriptionName); // create a function to use for checking the number of messages in replay Runnable checkLimit = () -> { @@ -2394,8 +2082,7 @@ public void testReadAheadLimit() throws Exception { for (int i = 0; i < numberOfKeys; i++) { String key = String.valueOf(i); byte[] keyBytes = key.getBytes(UTF_8); - int hash = StickyKeyConsumerSelector.makeStickyKeyHash(keyBytes); - if (dispatcher.getSelector().select(hash).consumerName().equals("c2")) { + if (dispatcher.getSelector().select(keyBytes).consumerName().equals("c2")) { keysForC2.add(key); } } @@ -2453,4 +2140,171 @@ public void testReadAheadLimit() throws Exception { }, Duration.ofSeconds(2), c1, c2, c3); assertEquals(remainingMessageValues, Collections.emptySet()); } + + @SneakyThrows + private StickyKeyConsumerSelector getSelector(String topic, String subscription) { + Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + return dispatcher.getSelector(); + } + + // This test case simulates a rolling restart scenario with behaviors that can trigger out-of-order issues. + // In earlier versions of Pulsar, this issue occurred in about 25% of cases. + // To increase the probability of reproducing the issue, use the invocationCount parameter. + @Test//(invocationCount = 50) + public void testOrderingAfterReconnects() throws Exception { + String topic = newUniqueName("testOrderingAfterReconnects"); + int numberOfKeys = 1000; + long pauseTime = 100L; + + @Cleanup + Producer producer = createProducer(topic, false); + + // create a consumer and close it to create a subscription + pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe() + .close(); + + Set remainingMessageValues = new HashSet<>(); + Map> keyPositions = new HashMap<>(); + BiFunction, Message, Boolean> messageHandler = (consumer, msg) -> { + synchronized (this) { + consumer.acknowledgeAsync(msg); + String key = msg.getKey(); + MessageIdAdv msgId = (MessageIdAdv) msg.getMessageId(); + Position currentPosition = PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId()); + Pair prevPair = keyPositions.get(key); + if (prevPair != null && prevPair.getLeft().compareTo(currentPosition) > 0) { + log.error("key: {} value: {} prev: {}/{} current: {}/{}", key, msg.getValue(), prevPair.getLeft(), + prevPair.getRight(), currentPosition, consumer.getConsumerName()); + fail("out of order"); + } + keyPositions.put(key, Pair.of(currentPosition, consumer.getConsumerName())); + boolean removed = remainingMessageValues.remove(msg.getValue()); + if (!removed) { + // duplicates are possible during reconnects, this is not an error + log.warn("Duplicate message: {} value: {}", msg.getMessageId(), msg.getValue()); + } + return true; + } + }; + + // Adding a new consumer. + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .startPaused(true) // start paused + .subscribe(); + + @Cleanup + Consumer c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(500) // use large receiver queue size + .subscribe(); + + @Cleanup + Consumer c3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .startPaused(true) // start paused + .subscribe(); + + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + + // find keys that will be assigned to c2 + List keysForC2 = new ArrayList<>(); + for (int i = 0; i < numberOfKeys; i++) { + String key = String.valueOf(i); + byte[] keyBytes = key.getBytes(UTF_8); + int hash = selector.makeStickyKeyHash(keyBytes); + if (selector.select(hash).consumerName().equals("c2")) { + keysForC2.add(key); + } + } + + // produce messages with keys that all get assigned to c2 + for (int i = 0; i < 1000; i++) { + String key = keysForC2.get(random.nextInt(keysForC2.size())); + //log.info("Producing message with key: {} value: {}", key, i); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + } + + Thread.sleep(2 * pauseTime); + // close c2 + c2.close(); + Thread.sleep(pauseTime); + // resume c1 and c3 + c1.resume(); + c3.resume(); + Thread.sleep(pauseTime); + // reconnect c2 + c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .subscribe(); + // close and reconnect c1 + c1.close(); + Thread.sleep(pauseTime); + c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .subscribe(); + // close and reconnect c3 + c3.close(); + Thread.sleep(pauseTime); + c3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .subscribe(); + + logTopicStats(topic); + + // produce more messages + for (int i = 1000; i < 2000; i++) { + String key = String.valueOf(random.nextInt(numberOfKeys)); + //log.info("Producing message with key: {} value: {}", key, i); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + } + + // consume the messages + receiveMessages(messageHandler, Duration.ofSeconds(2), c1, c2, c3); + + try { + assertEquals(remainingMessageValues, Collections.emptySet()); + } finally { + logTopicStats(topic); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java index 7889b19e5b29e..704af89777f05 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java @@ -18,18 +18,25 @@ */ package org.apache.pulsar.client.impl; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; +import static org.assertj.core.api.Assertions.assertThat; import com.google.common.collect.Sets; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.RandomStringUtils; +import lombok.SneakyThrows; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -40,7 +47,6 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.SubscriptionType; -import org.apache.pulsar.common.util.Murmur3_32Hash; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -65,33 +71,76 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + enum KeySharedSelectorType { + AutoSplit_ConsistentHashing(true), AutoSplit_Classic(true), Sticky(false); + final boolean autoSplit; + + KeySharedSelectorType(boolean autoSplit) { + this.autoSplit = autoSplit; + } + } + @DataProvider public Object[][] subType() { - return new Object[][] { { SubscriptionType.Shared }, { SubscriptionType.Key_Shared } }; + return new Object[][] { + { SubscriptionType.Shared, null }, + { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_ConsistentHashing }, + { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_Classic }, + { SubscriptionType.Key_Shared, KeySharedSelectorType.Sticky } + }; } - @Test(dataProvider = "subType") - public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType) + @Test(dataProvider = "subType", timeOut = 30000) + public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType, + KeySharedSelectorType selectorType) throws PulsarClientException { + if (selectorType == KeySharedSelectorType.AutoSplit_Classic) { + conf.setSubscriptionKeySharedUseConsistentHashing(false); + } + final int totalMsg = 1000; - String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5); - Map, List> nameToId = new ConcurrentHashMap<>(); + String topic = newUniqueName("broker-close-test"); + String subscriptionName = "sub-1"; + Map, List> unackedMessages = new ConcurrentHashMap<>(); Set pubMessages = Sets.newConcurrentHashSet(); Set recMessages = Sets.newConcurrentHashSet(); AtomicLong lastActiveTime = new AtomicLong(); AtomicBoolean canAcknowledgement = new AtomicBoolean(false); + if (subscriptionType == SubscriptionType.Key_Shared) { + // create and close consumer to create the dispatcher so that the selector can be used + ConsumerBuilder consumerBuilder = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName(subscriptionName) + .subscriptionType(subscriptionType); + if (subscriptionType == SubscriptionType.Key_Shared) { + if (selectorType.autoSplit) { + consumerBuilder.keySharedPolicy(KeySharedPolicy.autoSplitHashRange()); + } else { + consumerBuilder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(Range.of(0, 65535))); + } + } + consumerBuilder + .subscribe() + .close(); + } + List> consumerList = new ArrayList<>(); - // create 3 consumers - for (int i = 0; i < 3; i++) { + int consumerCount = 3; + + Range[] ranges = null; + if (subscriptionType == SubscriptionType.Key_Shared && !selectorType.autoSplit) { + ranges = splitRange(getSelector(topic, subscriptionName).getKeyHashRange(), consumerCount); + } + + for (int i = 0; i < consumerCount; i++) { ConsumerBuilder builder = pulsarClient.newConsumer() .topic(topic) - .subscriptionName("sub-1") + .consumerName("consumer-" + i) + .subscriptionName(subscriptionName) .subscriptionType(subscriptionType) .messageListener((consumer, msg) -> { lastActiveTime.set(System.currentTimeMillis()); - nameToId.computeIfAbsent(consumer, (k) -> new ArrayList<>()) - .add(msg.getMessageId()); recMessages.add(msg.getMessageId()); if (canAcknowledgement.get()) { try { @@ -99,19 +148,31 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc } catch (PulsarClientException e) { throw new RuntimeException(e); } + } else { + unackedMessages.computeIfAbsent(consumer, + (k) -> Collections.synchronizedList(new ArrayList<>())) + .add(msg.getMessageId()); } }); if (subscriptionType == SubscriptionType.Key_Shared) { - // ensure every consumer can be distributed messages - int hash = Murmur3_32Hash.getInstance().makeHash(("key-" + i).getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; - builder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(Range.of(hash, hash))); + if (selectorType.autoSplit) { + builder.keySharedPolicy(KeySharedPolicy.autoSplitHashRange()); + } else { + builder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(ranges[i])); + } } consumerList.add(builder.subscribe()); } + String[] keys = new String[consumerCount]; + for (int i = 0; i < consumerCount; i++) { + keys[i] = subscriptionType == SubscriptionType.Key_Shared ? + generateKeyForConsumer(getSelector(topic, subscriptionName), + consumerList.get(i).getConsumerName()) : "key-" + i; + } + Producer producer = pulsarClient.newProducer() .topic(topic) .enableBatching(true) @@ -122,42 +183,39 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc .create(); for (int i = 0; i < totalMsg; i++) { - byte[] msg = UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8); - producer.newMessage().key("key-" + (i % 3)).value(msg) + producer.newMessage() + .key(keys[i % consumerCount]) + .value(("message-" + i).getBytes(StandardCharsets.UTF_8)) .sendAsync().thenAccept(pubMessages::add); } + producer.flush(); + // Wait for all consumers can not read more messages. the consumers are stuck by max unacked messages. - Awaitility.await() - .pollDelay(5, TimeUnit.SECONDS) - .until(() -> - (System.currentTimeMillis() - lastActiveTime.get()) > TimeUnit.SECONDS.toMillis(5)); + waitUntilLastActiveTimeNoLongerGetsUpdated(lastActiveTime); // All consumers can acknowledge messages as they continue to receive messages. canAcknowledgement.set(true); // Acknowledgment of currently received messages to get out of stuck state due to unack message - for (Map.Entry, List> entry : nameToId.entrySet()) { + for (Map.Entry, List> entry : unackedMessages.entrySet()) { Consumer consumer = entry.getKey(); - consumer.acknowledge(entry.getValue()); + List messageIdList = entry.getValue(); + consumer.acknowledge(messageIdList); } + // refresh active time lastActiveTime.set(System.currentTimeMillis()); // Wait for all consumers to continue receiving messages. - Awaitility.await() - .atMost(30, TimeUnit.SECONDS) - .pollDelay(5, TimeUnit.SECONDS) - .until(() -> - (System.currentTimeMillis() - lastActiveTime.get()) > TimeUnit.SECONDS.toMillis(5)); + waitUntilLastActiveTimeNoLongerGetsUpdated(lastActiveTime); logTopicStats(topic); //Determine if all messages have been received. //If the dispatcher is stuck, we can not receive enough messages. Assert.assertEquals(totalMsg, pubMessages.size()); - Assert.assertEquals(recMessages.size(), pubMessages.size()); - Assert.assertTrue(recMessages.containsAll(pubMessages)); + assertThat(recMessages).containsExactlyInAnyOrderElementsOf(pubMessages); // cleanup producer.close(); @@ -165,4 +223,43 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc consumer.close(); } } + + private Range[] splitRange(Range keyHashRange, int consumerCount) { + Range[] ranges = new Range[consumerCount]; + int start = keyHashRange.getStart(); + for (int i = 0; i < consumerCount; i++) { + int end = Math.min(start + keyHashRange.size() / consumerCount, keyHashRange.getEnd()); + ranges[i] = Range.of(start, end); + start = end + 1; + } + return ranges; + } + + private String generateKeyForConsumer(StickyKeyConsumerSelector selector, + String consumerName) { + int i = 0; + while (!Thread.currentThread().isInterrupted()) { + String key = "key" + i++; + org.apache.pulsar.broker.service.Consumer selectedConsumer = selector.select(key.getBytes(UTF_8)); + if (selectedConsumer != null && selectedConsumer.consumerName().equals(consumerName)) { + return key; + } + } + return null; + } + + private static void waitUntilLastActiveTimeNoLongerGetsUpdated(AtomicLong lastActiveTime) { + Awaitility.await() + .pollInterval(100, TimeUnit.MILLISECONDS) + .until(() -> System.currentTimeMillis() - lastActiveTime.get() > TimeUnit.SECONDS.toMillis(1)); + } + + @SneakyThrows + private StickyKeyConsumerSelector getSelector(String topic, String subscription) { + Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + return dispatcher.getSelector(); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index e307e41862e74..7b7c1f5765cc5 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -118,15 +118,6 @@ public interface SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ String getKeySharedMode(); - /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ - Map getConsumersAfterMarkDeletePosition(); - - /** The last sent position of the cursor. This is for Key_Shared subscription. */ - String getLastSentPosition(); - - /** Set of individually sent ranges. This is for Key_Shared subscription. */ - String getIndividuallySentPositions(); - /** SubscriptionProperties (key/value strings) associated with this subscribe. */ Map getSubscriptionProperties(); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java index 488083f484b76..cbca1ef8f06bd 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java @@ -93,4 +93,12 @@ public int compareTo(Range o) { } return result; } + + public boolean contains(int value) { + return value >= start && value <= end; + } + + public int size() { + return end - start + 1; + } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java index 610c782518348..50168221fea37 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java @@ -61,4 +61,44 @@ public void testIntersect() { public void testInvalid() { Range.of(0, -5); } + + @Test + public void testCompareTo() { + Range range1 = Range.of(0, 5); + Range range2 = Range.of(0, 5); + Range range3 = Range.of(0, 10); + Range range4 = Range.of(5, 10); + + Assert.assertEquals(0, range1.compareTo(range2)); + Assert.assertTrue(range1.compareTo(range3) < 0); + Assert.assertTrue(range3.compareTo(range1) > 0); + Assert.assertTrue(range1.compareTo(range4) < 0); + Assert.assertTrue(range4.compareTo(range1) > 0); + } + + @Test + public void testContains() { + Range range = Range.of(0, 5); + + Assert.assertTrue(range.contains(0)); + Assert.assertTrue(range.contains(3)); + Assert.assertTrue(range.contains(5)); + Assert.assertFalse(range.contains(-1)); + Assert.assertFalse(range.contains(6)); + } + + @Test + public void testSize() { + Range range = Range.of(0, 0); + Assert.assertEquals(1, range.size()); + + range = Range.of(0, 1); + Assert.assertEquals(2, range.size()); + + range = Range.of(0, 5); + Assert.assertEquals(6, range.size()); + + range = Range.of(3, 3); + Assert.assertEquals(1, range.size()); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index 977ed28e86814..4206a4aa8d61b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.ArrayList; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -126,15 +125,9 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ public String keySharedMode; - /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ - public Map consumersAfterMarkDeletePosition; - /** The last sent position of the cursor. This is for Key_Shared subscription. */ public String lastSentPosition; - /** Set of individually sent ranges. This is for Key_Shared subscription. */ - public String individuallySentPositions; - /** The number of non-contiguous deleted messages ranges. */ public int nonContiguousDeletedMessagesRanges; @@ -160,7 +153,6 @@ public class SubscriptionStatsImpl implements SubscriptionStats { public SubscriptionStatsImpl() { this.consumers = new ArrayList<>(); - this.consumersAfterMarkDeletePosition = new LinkedHashMap<>(); this.subscriptionProperties = new HashMap<>(); this.bucketDelayedIndexStats = new HashMap<>(); } @@ -185,7 +177,6 @@ public void reset() { lastExpireTimestamp = 0L; lastMarkDeleteAdvancedTimestamp = 0L; consumers.clear(); - consumersAfterMarkDeletePosition.clear(); nonContiguousDeletedMessagesRanges = 0; nonContiguousDeletedMessagesRangesSerializedSize = 0; earliestMsgPublishTimeInBacklog = 0L; @@ -231,7 +222,6 @@ public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { } } this.allowOutOfOrderDelivery |= stats.allowOutOfOrderDelivery; - this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; if (this.earliestMsgPublishTimeInBacklog != 0 && stats.earliestMsgPublishTimeInBacklog != 0) { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 15b5676094ec1..8635368f00f0b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -1970,27 +1970,37 @@ public static MessageMetadata peekAndCopyMessageMetadata( return metadata; } - private static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); + public static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); + public static byte[] peekStickyKey(ByteBuf metadataAndPayload, String topic, String subscription) { try { int readerIdx = metadataAndPayload.readerIndex(); - MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); + MessageMetadata metadata = parseMessageMetadata(metadataAndPayload); metadataAndPayload.readerIndex(readerIdx); - if (metadata.hasOrderingKey()) { - return metadata.getOrderingKey(); - } else if (metadata.hasPartitionKey()) { - if (metadata.isPartitionKeyB64Encoded()) { - return Base64.getDecoder().decode(metadata.getPartitionKey()); - } - return metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); - } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { - String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); - return fallbackKey.getBytes(StandardCharsets.UTF_8); - } + return resolveStickyKey(metadata); } catch (Throwable t) { log.error("[{}] [{}] Failed to peek sticky key from the message metadata", topic, subscription, t); + return NONE_KEY; + } + } + + public static byte[] resolveStickyKey(MessageMetadata metadata) { + byte[] stickyKey; + if (metadata.hasOrderingKey()) { + stickyKey = metadata.getOrderingKey(); + } else if (metadata.hasPartitionKey()) { + if (metadata.isPartitionKeyB64Encoded()) { + stickyKey = Base64.getDecoder().decode(metadata.getPartitionKey()); + } else { + stickyKey = metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); + } + } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { + String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); + stickyKey = fallbackKey.getBytes(StandardCharsets.UTF_8); + } else { + stickyKey = NONE_KEY; } - return Commands.NONE_KEY; + return stickyKey; } public static int getCurrentProtocolVersion() { diff --git a/pulsar-common/src/main/resources/findbugsExclude.xml b/pulsar-common/src/main/resources/findbugsExclude.xml index df161c4b621a7..b3e511006bce3 100644 --- a/pulsar-common/src/main/resources/findbugsExclude.xml +++ b/pulsar-common/src/main/resources/findbugsExclude.xml @@ -53,4 +53,9 @@ + + + + + From 4d6dee456ca3c255d00d313995f7c1f9e78420f1 Mon Sep 17 00:00:00 2001 From: Jiawen Wang <74594733+summeriiii@users.noreply.github.com> Date: Wed, 9 Oct 2024 02:52:47 +0800 Subject: [PATCH 051/327] [fix][ml] Remove unnecessary return in ManagedLedgerImpl (#23418) --- .../apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 7 ------- 1 file changed, 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 cb19bd94bce01..c1081761b601f 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 @@ -2229,13 +2229,11 @@ public void readEntryComplete(Entry entry, Object ctx) { Object cbCtx = this.cntx; if (recycle(reOpCount)) { callback.readEntryComplete(entry, cbCtx); - return; } else { if (log.isDebugEnabled()) { log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); } entry.release(); - return; } } @@ -2246,7 +2244,6 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { Object cbCtx = this.cntx; if (recycle(reOpCount)) { callback.readEntryFailed(exception, cbCtx); - return; } else { if (log.isDebugEnabled()) { log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); @@ -2261,13 +2258,11 @@ public void readEntriesComplete(List returnedEntries, Object ctx) { Object cbCtx = this.cntx; if (recycle(reOpCount)) { callback.readEntriesComplete(returnedEntries, cbCtx); - return; } else { if (log.isDebugEnabled()) { log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); } returnedEntries.forEach(Entry::release); - return; } } @@ -2278,12 +2273,10 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { Object cbCtx = this.cntx; if (recycle(reOpCount)) { callback.readEntriesFailed(exception, cbCtx); - return; } else { if (log.isDebugEnabled()) { log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); } - return; } } From 9579c4dea9120ed29523d7eb56c1b0637aaa9bc4 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Tue, 8 Oct 2024 15:22:41 -0700 Subject: [PATCH 052/327] [improve][ci] Upgrade/Downgrade test (#22988) --- .github/workflows/pulsar-ci.yaml | 3 + build/run_integration_group.sh | 4 + .../containers/PulsarContainer.java | 5 + .../integration/topologies/PulsarCluster.java | 58 ++++-- .../topologies/PulsarClusterSpec.java | 6 + .../topologies/PulsarClusterTestBase.java | 6 +- .../upgrade/PulsarUpgradeDowngradeTest.java | 175 ++++++++++++++++++ .../src/test/resources/pulsar-upgrade.xml | 2 +- 8 files changed, 243 insertions(+), 16 deletions(-) create mode 100644 tests/integration/src/test/java/org/apache/pulsar/tests/integration/upgrade/PulsarUpgradeDowngradeTest.java diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 091dab25ec696..47a39bef9c908 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -603,6 +603,9 @@ jobs: - name: Metrics group: METRICS + - name: Upgrade + group: UPGRADE + steps: - name: checkout uses: actions/checkout@v4 diff --git a/build/run_integration_group.sh b/build/run_integration_group.sh index 2d82fce08878d..63b92d4e0a798 100755 --- a/build/run_integration_group.sh +++ b/build/run_integration_group.sh @@ -177,6 +177,10 @@ test_group_standalone() { mvn_run_integration_test "$@" -DintegrationTestSuiteFile=pulsar-standalone.xml -DintegrationTests } +test_group_upgrade() { + mvn_run_integration_test "$@" -DintegrationTestSuiteFile=pulsar-upgrade.xml -DintegrationTests +} + test_group_transaction() { mvn_run_integration_test "$@" -DintegrationTestSuiteFile=pulsar-transaction.xml -DintegrationTests } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java index 77cdc1bfd28a9..3cdb048aea55f 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java @@ -51,8 +51,13 @@ public abstract class PulsarContainer> exte public static final int BROKER_HTTP_PORT = 8080; public static final int BROKER_HTTPS_PORT = 8081; + public static final String ALPINE_IMAGE_NAME = "alpine:3.20"; public static final String DEFAULT_IMAGE_NAME = System.getenv().getOrDefault("PULSAR_TEST_IMAGE_NAME", "apachepulsar/pulsar-test-latest-version:latest"); + public static final String UPGRADE_TEST_IMAGE_NAME = System.getenv().getOrDefault("PULSAR_UPGRADE_TEST_IMAGE_NAME", + DEFAULT_IMAGE_NAME); + public static final String LAST_RELEASE_IMAGE_NAME = System.getenv().getOrDefault("PULSAR_LAST_RELEASE_IMAGE_NAME", + "apachepulsar/pulsar:3.0.7"); public static final String DEFAULT_HTTP_PATH = "/metrics"; public static final String PULSAR_2_5_IMAGE_NAME = "apachepulsar/pulsar:2.5.0"; public static final String PULSAR_2_4_IMAGE_NAME = "apachepulsar/pulsar:2.4.0"; diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java index 90f08a9639471..35fb453c4bb8e 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java @@ -72,22 +72,28 @@ public class PulsarCluster { * @return the built pulsar cluster */ public static PulsarCluster forSpec(PulsarClusterSpec spec) { + return forSpec(spec, Network.newNetwork()); + } + + public static PulsarCluster forSpec(PulsarClusterSpec spec, Network network) { + checkArgument(network != null, "Network should not be null"); CSContainer csContainer = null; if (!spec.enableOxia) { csContainer = new CSContainer(spec.clusterName) - .withNetwork(Network.newNetwork()) + .withNetwork(network) .withNetworkAliases(CSContainer.NAME); } - return new PulsarCluster(spec, csContainer, false); + return new PulsarCluster(spec, network, csContainer, false); } public static PulsarCluster forSpec(PulsarClusterSpec spec, CSContainer csContainer) { - return new PulsarCluster(spec, csContainer, true); + return new PulsarCluster(spec, csContainer.getNetwork(), csContainer, true); } @Getter private final PulsarClusterSpec spec; + public boolean closeNetworkOnExit = true; @Getter private final String clusterName; private final Network network; @@ -108,19 +114,18 @@ public static PulsarCluster forSpec(PulsarClusterSpec spec, CSContainer csContai private final String metadataStoreUrl; private final String configurationMetadataStoreUrl; - private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean sharedCsContainer) { - + private PulsarCluster(PulsarClusterSpec spec, Network network, CSContainer csContainer, boolean sharedCsContainer) { this.spec = spec; this.sharedCsContainer = sharedCsContainer; this.clusterName = spec.clusterName(); - if (csContainer != null ) { + if (network != null) { + this.network = network; + } else if (csContainer != null) { this.network = csContainer.getNetwork(); } else { this.network = Network.newNetwork(); } - - if (spec.enableOxia) { this.zkContainer = null; this.oxiaContainer = new OxiaContainer(clusterName); @@ -203,7 +208,9 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s .withEnv("PULSAR_PREFIX_diskUsageWarnThreshold", "0.95") .withEnv("diskUsageThreshold", "0.99") .withEnv("PULSAR_PREFIX_diskUsageLwmThreshold", "0.97") - .withEnv("nettyMaxFrameSizeBytes", String.valueOf(spec.maxMessageSize)); + .withEnv("nettyMaxFrameSizeBytes", String.valueOf(spec.maxMessageSize)) + .withEnv("ledgerDirectories", "data/bookkeeper/" + name + "/ledgers") + .withEnv("journalDirectory", "data/bookkeeper/" + name + "/journal"); if (spec.bookkeeperEnvs != null) { bookieContainer.withEnv(spec.bookkeeperEnvs); } @@ -262,10 +269,27 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s } )); + if (spec.dataContainer != null) { + if (!sharedCsContainer && csContainer != null) { + csContainer.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE); + } + if (zkContainer != null) { + zkContainer.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE); + } + proxyContainer.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE); + + bookieContainers.values().forEach(c -> c.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE)); + brokerContainers.values().forEach(c -> c.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE)); + workerContainers.values().forEach(c -> c.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE)); + } + spec.classPathVolumeMounts.forEach((key, value) -> { if (zkContainer != null) { zkContainer.withClasspathResourceMapping(key, value, BindMode.READ_WRITE); } + if (!sharedCsContainer && csContainer != null) { + csContainer.withClasspathResourceMapping(key, value, BindMode.READ_WRITE); + } proxyContainer.withClasspathResourceMapping(key, value, BindMode.READ_WRITE); bookieContainers.values().forEach(c -> c.withClasspathResourceMapping(key, value, BindMode.READ_WRITE)); @@ -323,6 +347,10 @@ public Map> getExternalServices() { } public void start() throws Exception { + start(true); + } + + public void start(boolean doInit) throws Exception { if (!spec.enableOxia) { // start the local zookeeper @@ -338,7 +366,7 @@ public void start() throws Exception { oxiaContainer.start(); } - { + if (doInit) { // Run cluster metadata initialization @Cleanup PulsarInitMetadataContainer init = new PulsarInitMetadataContainer( @@ -453,10 +481,12 @@ public synchronized void stop() { oxiaContainer.stop(); } - try { - network.close(); - } catch (Exception e) { - log.info("Failed to shutdown network for pulsar cluster {}", clusterName, e); + if (closeNetworkOnExit) { + try { + network.close(); + } catch (Exception e) { + log.info("Failed to shutdown network for pulsar cluster {}", clusterName, e); + } } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java index 8a991be49fad0..ca45c9b7c9b82 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java @@ -124,6 +124,12 @@ public class PulsarClusterSpec { @Builder.Default Map classPathVolumeMounts = new TreeMap<>(); + /** + * Data container + */ + @Builder.Default + GenericContainer dataContainer = null; + /** * Pulsar Test Image Name * diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterTestBase.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterTestBase.java index 93e2221ab2493..8b99f21373560 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterTestBase.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterTestBase.java @@ -142,6 +142,10 @@ protected void beforeStartCluster() throws Exception { } protected void setupCluster(PulsarClusterSpec spec) throws Exception { + setupCluster(spec, true); + } + + protected void setupCluster(PulsarClusterSpec spec, boolean doInit) throws Exception { incrementSetupNumber(); log.info("Setting up cluster {} with {} bookies, {} brokers", spec.clusterName(), spec.numBookies(), spec.numBrokers()); @@ -150,7 +154,7 @@ protected void setupCluster(PulsarClusterSpec spec) throws Exception { beforeStartCluster(); - pulsarCluster.start(); + pulsarCluster.start(doInit); pulsarAdmin = PulsarAdmin.builder().serviceHttpUrl(pulsarCluster.getHttpServiceUrl()).build(); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/upgrade/PulsarUpgradeDowngradeTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/upgrade/PulsarUpgradeDowngradeTest.java new file mode 100644 index 0000000000000..ddabd67b2294b --- /dev/null +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/upgrade/PulsarUpgradeDowngradeTest.java @@ -0,0 +1,175 @@ +/* + * 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.tests.integration.upgrade; + +import com.github.dockerjava.api.model.Bind; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.tests.integration.containers.PulsarContainer; +import org.apache.pulsar.tests.integration.topologies.PulsarCluster; +import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; +import org.apache.pulsar.tests.integration.topologies.PulsarClusterTestBase; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testng.annotations.Test; +import java.util.stream.Stream; +import static java.util.stream.Collectors.joining; +import static org.testng.Assert.assertEquals; + +/** + * Test upgrading/downgrading Pulsar cluster from major releases. + */ +@Slf4j +public class PulsarUpgradeDowngradeTest extends PulsarClusterTestBase { + + @Test(timeOut=600_000) + public void upgradeTest() throws Exception { + testUpgradeDowngrade(PulsarContainer.LAST_RELEASE_IMAGE_NAME, PulsarContainer.UPGRADE_TEST_IMAGE_NAME); + } + + private void testUpgradeDowngrade(String imageOld, String imageNew) throws Exception { + final String clusterName = Stream.of(this.getClass().getSimpleName(), randomName(5)) + .filter(s -> !s.isEmpty()) + .collect(joining("-")); + String topicName = generateTopicName("testupdown", true); + + @Cleanup + Network network = Network.newNetwork(); + @Cleanup + GenericContainer alpine = new GenericContainer<>(PulsarContainer.ALPINE_IMAGE_NAME) + .withExposedPorts(80) + .withNetwork(network) + .withNetworkAliases("shared-storage") + .withEnv("MAGIC_NUMBER", "42") + .withCreateContainerCmdModifier(createContainerCmd -> createContainerCmd + .getHostConfig() + .withBinds(Bind.parse("/pulsar/data:/pulsar/data"))) + .withCommand("/bin/sh", "-c", + "mkdir -p /pulsar/data && " + + "chmod -R ug+rwx /pulsar/data && " + + "chown -R 10000:0 /pulsar/data && " + + "rm -rf /pulsar/data/* && " + + "while true; do echo \"$MAGIC_NUMBER\" | nc -l -p 80; done"); + alpine.start(); + + PulsarClusterSpec specOld = PulsarClusterSpec.builder() + .numBookies(2) + .numBrokers(1) + .clusterName(clusterName) + .dataContainer(alpine) + .pulsarTestImage(imageOld) + .build(); + + PulsarClusterSpec specNew = PulsarClusterSpec.builder() + .numBookies(2) + .numBrokers(1) + .clusterName(clusterName) + .dataContainer(alpine) + .pulsarTestImage(imageNew) + .build(); + + log.info("Setting up OLD cluster {} with {} bookies, {} brokers using {}", + specOld.clusterName(), specOld.numBookies(), specOld.numBrokers(), imageOld); + + pulsarCluster = PulsarCluster.forSpec(specNew, network); + pulsarCluster.closeNetworkOnExit = false; + pulsarCluster.start(true); + + try { + log.info("setting retention"); + pulsarCluster.runAdminCommandOnAnyBroker("namespaces", + "set-retention", "--size", "100M", "--time", "100m", "public/default"); + + publishAndConsume(topicName, pulsarCluster.getPlainTextServiceUrl(), 10, 10); + } finally { + pulsarCluster.stop(); + } + + log.info("Upgrading to NEW cluster {} with {} bookies, {} brokers using {}", + specNew.clusterName(), specNew.numBookies(), specNew.numBrokers(), imageNew); + + pulsarCluster = PulsarCluster.forSpec(specNew, network); + pulsarCluster.closeNetworkOnExit = false; + pulsarCluster.start(false); + + try { + publishAndConsume(topicName, pulsarCluster.getPlainTextServiceUrl(), 10, 20); + } finally { + pulsarCluster.stop(); + } + + log.info("Downgrading to OLD cluster {} with {} bookies, {} brokers using {}", + specOld.clusterName(), specOld.numBookies(), specOld.numBrokers(), imageOld); + + pulsarCluster = PulsarCluster.forSpec(specOld, network); + pulsarCluster.closeNetworkOnExit = false; + pulsarCluster.start(false); + + try { + publishAndConsume(topicName, pulsarCluster.getPlainTextServiceUrl(), 10, 30); + } finally { + pulsarCluster.stop(); + alpine.stop(); + network.close(); + } + } + + private void publishAndConsume(String topicName, String serviceUrl, int numProduce, int numConsume) throws Exception { + log.info("publishAndConsume: topic name: {}", topicName); + + @Cleanup + PulsarClient client = PulsarClient.builder() + .serviceUrl(serviceUrl) + .build(); + + @Cleanup + Producer producer = client.newProducer(Schema.STRING) + .topic(topicName) + .create(); + + log.info("Publishing {} messages", numProduce); + for (int i = numConsume - numProduce; i < numConsume; i++) { + log.info("Publishing message: {}", "smoke-message-" + i); + producer.send("smoke-message-" + i); + } + + @Cleanup + Consumer consumer = client.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName("my-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + consumer.seek(MessageId.earliest); + + log.info("Consuming {} messages", numConsume); + for (int i = 0; i < numConsume; i++) { + log.info("Waiting for message: {}", i); + Message m = consumer.receive(); + log.info("Received message: {}", m.getValue()); + assertEquals("smoke-message-" + i, m.getValue()); + } + } +} diff --git a/tests/integration/src/test/resources/pulsar-upgrade.xml b/tests/integration/src/test/resources/pulsar-upgrade.xml index a52db54753372..dc966b160ba17 100644 --- a/tests/integration/src/test/resources/pulsar-upgrade.xml +++ b/tests/integration/src/test/resources/pulsar-upgrade.xml @@ -22,7 +22,7 @@ - + From 84b834f95c83e2385d8ca9bccb4cb78120ed582c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 9 Oct 2024 02:34:21 +0300 Subject: [PATCH 053/327] [improve][broker] PIP-379: Snapshot hash range assignments only in AUTO_SPLIT ordered mode (#23423) --- ...stentHashingStickyKeyConsumerSelector.java | 35 ++++++++++++++----- ...ngeAutoSplitStickyKeyConsumerSelector.java | 32 ++++++++++++----- ...ngeExclusiveStickyKeyConsumerSelector.java | 25 ++++--------- .../service/StickyKeyConsumerSelector.java | 5 +-- ...tStickyKeyDispatcherMultipleConsumers.java | 10 +++--- ...tHashingStickyKeyConsumerSelectorTest.java | 6 ++-- ...utoSplitStickyKeyConsumerSelectorTest.java | 6 ++-- 7 files changed, 73 insertions(+), 46 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java index 8381f9543bdc2..fde140a299c27 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.Optional; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.locks.ReadWriteLock; @@ -44,21 +45,32 @@ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyCons private final int numberOfPoints; private final Range keyHashRange; + private final boolean addOrRemoveReturnsImpactedConsumersResult; private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints) { - this(numberOfPoints, DEFAULT_RANGE_SIZE - 1); + this(numberOfPoints, false); } - public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints, int rangeMaxValue) { + public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints, + boolean addOrRemoveReturnsImpactedConsumersResult) { + this(numberOfPoints, addOrRemoveReturnsImpactedConsumersResult, DEFAULT_RANGE_SIZE - 1); + } + + public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints, + boolean addOrRemoveReturnsImpactedConsumersResult, + int rangeMaxValue) { + this.addOrRemoveReturnsImpactedConsumersResult = addOrRemoveReturnsImpactedConsumersResult; this.hashRing = new TreeMap<>(); this.numberOfPoints = numberOfPoints; this.keyHashRange = Range.of(STICKY_KEY_HASH_NOT_SET + 1, rangeMaxValue); - this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); + this.consumerHashAssignmentsSnapshot = addOrRemoveReturnsImpactedConsumersResult + ? ConsumerHashAssignmentsSnapshot.empty() + : null; } @Override - public CompletableFuture addConsumer(Consumer consumer) { + public CompletableFuture> addConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); @@ -76,11 +88,14 @@ public CompletableFuture addConsumer(Consumer consumer) consumerNameIndexTracker.decreaseConsumerRefCount(removed); } } + if (!addOrRemoveReturnsImpactedConsumersResult) { + return CompletableFuture.completedFuture(Optional.empty()); + } ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); ImpactedConsumersResult impactedConsumers = consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); consumerHashAssignmentsSnapshot = assignmentsAfter; - return CompletableFuture.completedFuture(impactedConsumers); + return CompletableFuture.completedFuture(Optional.of(impactedConsumers)); } finally { rwLock.writeLock().unlock(); } @@ -103,7 +118,7 @@ private int calculateHashForConsumerAndIndex(Consumer consumer, int consumerName } @Override - public ImpactedConsumersResult removeConsumer(Consumer consumer) { + public Optional removeConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); @@ -117,11 +132,14 @@ public ImpactedConsumersResult removeConsumer(Consumer consumer) { } } } + if (!addOrRemoveReturnsImpactedConsumersResult) { + return Optional.empty(); + } ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); ImpactedConsumersResult impactedConsumers = consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); consumerHashAssignmentsSnapshot = assignmentsAfter; - return impactedConsumers; + return Optional.of(impactedConsumers); } finally { rwLock.writeLock().unlock(); } @@ -155,7 +173,8 @@ public Range getKeyHashRange() { public ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { rwLock.readLock().lock(); try { - return consumerHashAssignmentsSnapshot; + return consumerHashAssignmentsSnapshot != null ? consumerHashAssignmentsSnapshot + : internalGetConsumerHashAssignmentsSnapshot(); } finally { rwLock.readLock().unlock(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java index b90aef739f2b1..48d5491d119b2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListMap; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; @@ -59,13 +60,20 @@ public class HashRangeAutoSplitStickyKeyConsumerSelector implements StickyKeyCon private final Range keyHashRange; private final ConcurrentSkipListMap rangeMap; private final Map consumerRange; + private final boolean addOrRemoveReturnsImpactedConsumersResult; private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public HashRangeAutoSplitStickyKeyConsumerSelector() { - this(DEFAULT_RANGE_SIZE); + this(false); } - public HashRangeAutoSplitStickyKeyConsumerSelector(int rangeSize) { + public HashRangeAutoSplitStickyKeyConsumerSelector(boolean addOrRemoveReturnsImpactedConsumersResult) { + this(DEFAULT_RANGE_SIZE, addOrRemoveReturnsImpactedConsumersResult); + } + + public HashRangeAutoSplitStickyKeyConsumerSelector(int rangeSize, + boolean addOrRemoveReturnsImpactedConsumersResult) { + this.addOrRemoveReturnsImpactedConsumersResult = addOrRemoveReturnsImpactedConsumersResult; if (rangeSize < 2) { throw new IllegalArgumentException("range size must greater than 2"); } @@ -76,11 +84,12 @@ public HashRangeAutoSplitStickyKeyConsumerSelector(int rangeSize) { this.consumerRange = new HashMap<>(); this.rangeSize = rangeSize; this.keyHashRange = Range.of(0, rangeSize - 1); - this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); + this.consumerHashAssignmentsSnapshot = addOrRemoveReturnsImpactedConsumersResult + ? ConsumerHashAssignmentsSnapshot.empty() : null; } @Override - public synchronized CompletableFuture addConsumer(Consumer consumer) { + public synchronized CompletableFuture> addConsumer(Consumer consumer) { if (rangeMap.isEmpty()) { rangeMap.put(rangeSize, consumer); consumerRange.put(consumer, rangeSize); @@ -91,15 +100,18 @@ public synchronized CompletableFuture addConsumer(Consu return CompletableFuture.failedFuture(e); } } + if (!addOrRemoveReturnsImpactedConsumersResult) { + return CompletableFuture.completedFuture(Optional.empty()); + } ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); ImpactedConsumersResult impactedConsumers = consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); consumerHashAssignmentsSnapshot = assignmentsAfter; - return CompletableFuture.completedFuture(impactedConsumers); + return CompletableFuture.completedFuture(Optional.of(impactedConsumers)); } @Override - public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { + public synchronized Optional removeConsumer(Consumer consumer) { Integer removeRange = consumerRange.remove(consumer); if (removeRange != null) { if (removeRange == rangeSize && rangeMap.size() > 1) { @@ -111,11 +123,14 @@ public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { rangeMap.remove(removeRange); } } + if (!addOrRemoveReturnsImpactedConsumersResult) { + return Optional.empty(); + } ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); ImpactedConsumersResult impactedConsumers = consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); consumerHashAssignmentsSnapshot = assignmentsAfter; - return impactedConsumers; + return Optional.of(impactedConsumers); } @Override @@ -134,7 +149,8 @@ public Range getKeyHashRange() { @Override public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { - return consumerHashAssignmentsSnapshot; + return consumerHashAssignmentsSnapshot != null ? consumerHashAssignmentsSnapshot + : internalGetConsumerHashAssignmentsSnapshot(); } private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java index 7c76d9dca7456..904fb702a943e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListMap; import org.apache.pulsar.client.api.Range; @@ -38,7 +39,6 @@ public class HashRangeExclusiveStickyKeyConsumerSelector implements StickyKeyCon private final int rangeSize; private final Range keyHashRange; private final ConcurrentSkipListMap rangeMap; - private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public HashRangeExclusiveStickyKeyConsumerSelector() { this(DEFAULT_RANGE_SIZE); @@ -52,11 +52,10 @@ public HashRangeExclusiveStickyKeyConsumerSelector(int rangeSize) { this.rangeSize = rangeSize; this.keyHashRange = Range.of(0, rangeSize - 1); this.rangeMap = new ConcurrentSkipListMap<>(); - this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); } @Override - public synchronized CompletableFuture addConsumer(Consumer consumer) { + public synchronized CompletableFuture> addConsumer(Consumer consumer) { return validateKeySharedMeta(consumer).thenApply(__ -> { try { return internalAddConsumer(consumer); @@ -66,7 +65,7 @@ public synchronized CompletableFuture addConsumer(Consu }); } - private synchronized ImpactedConsumersResult internalAddConsumer(Consumer consumer) + private synchronized Optional internalAddConsumer(Consumer consumer) throws BrokerServiceException.ConsumerAssignException { Consumer conflictingConsumer = findConflictingConsumer(consumer.getKeySharedMeta().getHashRangesList()); if (conflictingConsumer != null) { @@ -77,29 +76,17 @@ private synchronized ImpactedConsumersResult internalAddConsumer(Consumer consum rangeMap.put(intRange.getStart(), consumer); rangeMap.put(intRange.getEnd(), consumer); } - ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); - ImpactedConsumersResult impactedConsumers = - consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); - consumerHashAssignmentsSnapshot = assignmentsAfter; - return impactedConsumers; + return Optional.empty(); } @Override - public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { + public synchronized Optional removeConsumer(Consumer consumer) { rangeMap.entrySet().removeIf(entry -> entry.getValue().equals(consumer)); - ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); - ImpactedConsumersResult impactedConsumers = - consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); - consumerHashAssignmentsSnapshot = assignmentsAfter; - return impactedConsumers; + return Optional.empty(); } @Override public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { - return consumerHashAssignmentsSnapshot; - } - - private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { List result = new ArrayList<>(); Map.Entry prev = null; for (Map.Entry entry: rangeMap.entrySet()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java index 1ead3f946c24d..099929fd2a696 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.api.Range; @@ -50,7 +51,7 @@ public interface StickyKeyConsumerSelector { * The result contains information about the existing consumers whose hash ranges were affected * by the addition of the new consumer. */ - CompletableFuture addConsumer(Consumer consumer); + CompletableFuture> addConsumer(Consumer consumer); /** * Remove the consumer. @@ -59,7 +60,7 @@ public interface StickyKeyConsumerSelector { * @return the result of impacted consumers. The result contains information about the existing consumers * whose hash ranges were affected by the removal of the consumer. */ - ImpactedConsumersResult removeConsumer(Consumer consumer); + Optional removeConsumer(Consumer consumer); /** * Select a consumer by sticky key. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index a78e4e46c0e5a..925e99ed699a2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -91,9 +91,9 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi case AUTO_SPLIT: if (conf.isSubscriptionKeySharedUseConsistentHashing()) { selector = new ConsistentHashingStickyKeyConsumerSelector( - conf.getSubscriptionKeySharedConsistentHashingReplicaPoints()); + conf.getSubscriptionKeySharedConsistentHashingReplicaPoints(), drainingHashesRequired); } else { - selector = new HashRangeAutoSplitStickyKeyConsumerSelector(); + selector = new HashRangeAutoSplitStickyKeyConsumerSelector(drainingHashesRequired); } break; case STICKY: @@ -155,7 +155,7 @@ public void endBatch() { drainingHashesTracker.endBatch(); } }); - registerDrainingHashes(consumer, impactedConsumers); + registerDrainingHashes(consumer, impactedConsumers.orElseThrow()); } }).exceptionally(ex -> { internalRemoveConsumer(consumer); @@ -184,13 +184,13 @@ private synchronized void registerDrainingHashes(Consumer skipConsumer, @Override public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { // The consumer must be removed from the selector before calling the superclass removeConsumer method. - ImpactedConsumersResult impactedConsumers = selector.removeConsumer(consumer); + Optional impactedConsumers = selector.removeConsumer(consumer); super.removeConsumer(consumer); if (drainingHashesRequired) { // register draining hashes for the impacted consumers and ranges, in case a hash switched from one // consumer to another. This will handle the case where a hash gets switched from an existing // consumer to another existing consumer during removal. - registerDrainingHashes(consumer, impactedConsumers); + registerDrainingHashes(consumer, impactedConsumers.orElseThrow()); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java index e2feb2050652b..2b01256611b01 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java @@ -476,7 +476,8 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() { @Test public void testShouldContainMinimalMappingChangesWhenConsumerLeavesAndRejoins() { - final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final ConsistentHashingStickyKeyConsumerSelector selector = + new ConsistentHashingStickyKeyConsumerSelector(100, true); final String consumerName = "consumer"; final int numOfInitialConsumers = 10; List consumers = new ArrayList<>(); @@ -563,7 +564,8 @@ public void testPerformanceOfAdding1000ConsumersWith100Points() { // test that adding 1000 consumers with 100 points runs in a reasonable time. // This takes about 1 second on Apple M3 // this unit test can be used for basic profiling - final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final ConsistentHashingStickyKeyConsumerSelector selector = + new ConsistentHashingStickyKeyConsumerSelector(100, true); for (int i = 0; i < 1000; i++) { // use real class to avoid Mockito over head final Consumer consumer = new Consumer("consumer" + i, 0) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelectorTest.java index 98e27ebb9fb83..61fc015cf953e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelectorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelectorTest.java @@ -37,7 +37,8 @@ public class HashRangeAutoSplitStickyKeyConsumerSelectorTest { @Test public void testGetConsumerKeyHashRanges() throws BrokerServiceException.ConsumerAssignException { - HashRangeAutoSplitStickyKeyConsumerSelector selector = new HashRangeAutoSplitStickyKeyConsumerSelector(2 << 5); + HashRangeAutoSplitStickyKeyConsumerSelector selector = + new HashRangeAutoSplitStickyKeyConsumerSelector(2 << 5, false); List consumerName = Arrays.asList("consumer1", "consumer2", "consumer3", "consumer4"); List consumers = new ArrayList<>(); for (String s : consumerName) { @@ -61,7 +62,8 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume @Test public void testGetConsumerKeyHashRangesWithSameConsumerName() throws Exception { - HashRangeAutoSplitStickyKeyConsumerSelector selector = new HashRangeAutoSplitStickyKeyConsumerSelector(2 << 5); + HashRangeAutoSplitStickyKeyConsumerSelector selector = + new HashRangeAutoSplitStickyKeyConsumerSelector(2 << 5, false); final String consumerName = "My-consumer"; List consumers = new ArrayList<>(); for (int i = 0; i < 3; i++) { From 5aadec02a3e8767f55d4101a2efde47a86094e68 Mon Sep 17 00:00:00 2001 From: psxjoy Date: Wed, 9 Oct 2024 12:29:13 +0800 Subject: [PATCH 054/327] [fix][broker] Fix typos in pulsar-broker and tiered-storage. (#23415) --- .../java/org/apache/pulsar/broker/web/WebServiceTest.java | 8 ++++---- .../provider/AbstractJCloudBlobStoreFactoryTest.java | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index 08041d72c7e44..e6e792c2f3839 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -176,7 +176,7 @@ public void testDefaultClientVersion() throws Exception { } /** - * Test that if enableTls option is enabled, WebServcie is available both on HTTP and HTTPS. + * Test that if enableTls option is enabled, WebService is available both on HTTP and HTTPS. * * @throws Exception */ @@ -198,7 +198,7 @@ public void testTlsEnabled() throws Exception { } /** - * Test that if enableTls option is disabled, WebServcie is available only on HTTP. + * Test that if enableTls option is disabled, WebService is available only on HTTP. * * @throws Exception */ @@ -221,7 +221,7 @@ public void testTlsDisabled() throws Exception { } /** - * Test that if enableAuth option and allowInsecure option are enabled, WebServcie requires trusted/untrusted client + * Test that if enableAuth option and allowInsecure option are enabled, WebService requires trusted/untrusted client * certificate. * * @throws Exception @@ -245,7 +245,7 @@ public void testTlsAuthAllowInsecure() throws Exception { } /** - * Test that if enableAuth option is enabled, WebServcie requires trusted client certificate. + * Test that if enableAuth option is enabled, WebService requires trusted client certificate. * * @throws Exception */ diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/provider/AbstractJCloudBlobStoreFactoryTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/provider/AbstractJCloudBlobStoreFactoryTest.java index 17a337cc22a3c..93fb6dcc8d52e 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/provider/AbstractJCloudBlobStoreFactoryTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/provider/AbstractJCloudBlobStoreFactoryTest.java @@ -98,9 +98,9 @@ protected void sendMultipartPayload(String containerName, String blobName, Strin blobStore.completeMultipartUpload(mpu, parts); } - protected void deleteBlobAndVerify(String conatinerName, String blobName) { - blobStore.removeBlob(conatinerName, blobName); - Assert.assertFalse(blobStore.blobExists(conatinerName, blobName)); + protected void deleteBlobAndVerify(String containerName, String blobName) { + blobStore.removeBlob(containerName, blobName); + Assert.assertFalse(blobStore.blobExists(containerName, blobName)); } protected void deleteContainerAndVerify(String containerName) { From 676fdb1ffb4392bb7b10b8d1e8ba94b379b25166 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 9 Oct 2024 08:38:14 +0300 Subject: [PATCH 055/327] [improve][broker] PIP-379: Enable the use of the classic implementation of Key_Shared / Shared with feature flag (#23424) --- .../pulsar/broker/ServiceConfiguration.java | 16 + .../AbstractDelayedDeliveryTracker.java | 8 +- .../BucketDelayedDeliveryTrackerFactory.java | 6 +- .../DelayedDeliveryTrackerFactory.java | 4 +- .../InMemoryDelayedDeliveryTracker.java | 13 +- ...InMemoryDelayedDeliveryTrackerFactory.java | 6 +- .../bucket/BucketDelayedDeliveryTracker.java | 6 +- .../AbstractDispatcherMultipleConsumers.java | 6 - .../pulsar/broker/service/BrokerService.java | 16 +- .../pulsar/broker/service/Consumer.java | 10 +- .../pulsar/broker/service/Dispatcher.java | 4 +- .../broker/service/StickyKeyDispatcher.java | 48 + ...PersistentDispatcherMultipleConsumers.java | 67 + .../MessageRedeliveryController.java | 8 +- ...PersistentDispatcherMultipleConsumers.java | 25 +- ...entDispatcherMultipleConsumersClassic.java | 1374 +++++++++++++++++ ...tStickyKeyDispatcherMultipleConsumers.java | 4 +- ...KeyDispatcherMultipleConsumersClassic.java | 583 +++++++ .../persistent/PersistentSubscription.java | 50 +- .../service/persistent/PersistentTopic.java | 16 +- .../apache/pulsar/broker/BrokerTestUtil.java | 25 +- .../delayed/AbstractDeliveryTrackerTest.java | 4 +- .../DelayedDeliveryTrackerFactoryTest.java | 14 +- .../delayed/InMemoryDeliveryTrackerTest.java | 7 +- .../BucketDelayedDeliveryTrackerTest.java | 4 +- .../broker/service/BatchMessageTest.java | 4 +- .../BatchMessageWithBatchIndexLevelTest.java | 16 +- ...sistentDispatcherFailoverConsumerTest.java | 15 +- .../broker/service/PersistentTopicTest.java | 3 +- .../persistent/BucketDelayedDeliveryTest.java | 12 +- .../persistent/DelayedDeliveryTest.java | 8 +- ...ispatcherMultipleConsumersClassicTest.java | 172 +++ ...istentDispatcherMultipleConsumersTest.java | 2 +- ...ispatcherMultipleConsumersClassicTest.java | 482 ++++++ ...ckyKeyDispatcherMultipleConsumersTest.java | 2 +- .../broker/stats/ConsumerStatsTest.java | 2 +- .../client/api/KeySharedSubscriptionTest.java | 281 ++-- ...criptionMessageDispatchThrottlingTest.java | 18 +- ...SubscriptionPauseOnAckStatPersistTest.java | 5 +- ...edSubscriptionMaxUnackedMessagesTest.java} | 42 +- .../tests/KeySharedImplementationType.java | 61 + .../common/policies/data/ConsumerStats.java | 4 +- .../policies/data/SubscriptionStats.java | 3 + .../data/stats/ConsumerStatsImpl.java | 10 +- .../data/stats/SubscriptionStatsImpl.java | 8 +- 45 files changed, 3211 insertions(+), 263 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java rename pulsar-broker/src/test/java/org/apache/pulsar/client/impl/{KeySharedSubscriptionTest.java => KeySharedSubscriptionMaxUnackedMessagesTest.java} (85%) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 1b021bd569969..19e9ff625cada 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -817,6 +817,22 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + "The higher the number, the more equal the assignment of keys to consumers") private int subscriptionKeySharedConsistentHashingReplicaPoints = 100; + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For persistent Key_Shared subscriptions, enables the use of the classic implementation of the " + + "Key_Shared subscription that was used before Pulsar 4.0.0 and PIP-379.", + dynamic = true + ) + private boolean subscriptionKeySharedUseClassicPersistentImplementation = false; + + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For persistent Shared subscriptions, enables the use of the classic implementation of the Shared " + + "subscription that was used before Pulsar 4.0.0.", + dynamic = true + ) + private boolean subscriptionSharedUseClassicPersistentImplementation = false; + @FieldContext( category = CATEGORY_POLICIES, doc = "Set the default behavior for message deduplication in the broker.\n\n" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java index f93a627bca7b8..bec5134c4f79a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java @@ -24,12 +24,12 @@ import java.time.Clock; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; @Slf4j public abstract class AbstractDelayedDeliveryTracker implements DelayedDeliveryTracker, TimerTask { - protected final PersistentDispatcherMultipleConsumers dispatcher; + protected final AbstractPersistentDispatcherMultipleConsumers dispatcher; // Reference to the shared (per-broker) timer for delayed delivery protected final Timer timer; @@ -49,13 +49,13 @@ public abstract class AbstractDelayedDeliveryTracker implements DelayedDeliveryT private final boolean isDelayedDeliveryDeliverAtTimeStrict; - public AbstractDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, + public AbstractDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict) { this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict); } - public AbstractDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, + public AbstractDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, Clock clock, boolean isDelayedDeliveryDeliverAtTimeStrict) { this.dispatcher = dispatcher; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java index 11ad243e0c9d1..c2d002ad19cb0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java @@ -36,7 +36,7 @@ import org.apache.pulsar.broker.delayed.bucket.BucketSnapshotStorage; import org.apache.pulsar.broker.delayed.bucket.RecoverDelayedDeliveryTrackerException; import org.apache.pulsar.broker.service.BrokerService; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,7 +78,7 @@ public void initialize(PulsarService pulsarService) throws Exception { } @Override - public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { + public DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher) { String topicName = dispatcher.getTopic().getName(); String subscriptionName = dispatcher.getSubscription().getName(); BrokerService brokerService = dispatcher.getTopic().getBrokerService(); @@ -97,7 +97,7 @@ public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers d } @VisibleForTesting - BucketDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers dispatcher) + BucketDelayedDeliveryTracker newTracker0(AbstractPersistentDispatcherMultipleConsumers dispatcher) throws RecoverDelayedDeliveryTrackerException { return new BucketDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java index 5427a46a2e4b3..763b6d66da142 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java @@ -20,7 +20,7 @@ import com.google.common.annotations.Beta; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; /** * Factory of InMemoryDelayedDeliveryTracker objects. This is the entry point for implementations. @@ -42,7 +42,7 @@ public interface DelayedDeliveryTrackerFactory extends AutoCloseable { * @param dispatcher * a multi-consumer dispatcher instance */ - DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher); + DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher); /** * Close the factory and release all the resources. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index 8bd9fafa13715..bdc6e4c814e33 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -28,7 +28,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; @Slf4j @@ -52,17 +52,18 @@ public class InMemoryDelayedDeliveryTracker extends AbstractDelayedDeliveryTrack // Track whether we have seen all messages with fixed delay so far. private boolean messagesHaveFixedDelay = true; - InMemoryDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, + InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, + long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, long fixedDelayDetectionLookahead) { this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); } - public InMemoryDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, - long tickTimeMillis, Clock clock, - boolean isDelayedDeliveryDeliverAtTimeStrict, - long fixedDelayDetectionLookahead) { + public InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, + long tickTimeMillis, Clock clock, + boolean isDelayedDeliveryDeliverAtTimeStrict, + long fixedDelayDetectionLookahead) { super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); this.fixedDelayDetectionLookahead = fixedDelayDetectionLookahead; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java index 179cf74db4179..f8b8f5a8ba459 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java @@ -25,7 +25,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,7 +51,7 @@ public void initialize(PulsarService pulsarService) { } @Override - public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { + public DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher) { String topicName = dispatcher.getTopic().getName(); String subscriptionName = dispatcher.getSubscription().getName(); DelayedDeliveryTracker tracker = DelayedDeliveryTracker.DISABLE; @@ -66,7 +66,7 @@ public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers d } @VisibleForTesting - InMemoryDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers dispatcher) { + InMemoryDelayedDeliveryTracker newTracker0(AbstractPersistentDispatcherMultipleConsumers dispatcher) { return new InMemoryDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index 47c78fa9ee2ec..0091bf5b9bd30 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -57,7 +57,7 @@ import org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.proto.DelayedIndex; import org.apache.pulsar.broker.delayed.proto.SnapshotSegment; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; @@ -105,7 +105,7 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker private CompletableFuture pendingLoad = null; - public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, + public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, BucketSnapshotStorage bucketSnapshotStorage, @@ -117,7 +117,7 @@ public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispat maxIndexesPerBucketSnapshotSegment, maxNumBuckets); } - public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, + public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, Clock clock, boolean isDelayedDeliveryDeliverAtTimeStrict, BucketSnapshotStorage bucketSnapshotStorage, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java index 9fc6b9581a3ac..e3c2cf40cf318 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java @@ -24,10 +24,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * @@ -239,7 +236,4 @@ private int getFirstConsumerIndexOfPriority(int targetPriority) { return -1; } - private static final Logger log = LoggerFactory.getLogger(PersistentStickyKeyDispatcherMultipleConsumers.class); - - } 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 8d0b9a4a84e6a..fee5e25647ce6 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 @@ -129,8 +129,8 @@ import org.apache.pulsar.broker.service.TopicEventsListener.EventStage; import org.apache.pulsar.broker.service.TopicEventsListener.TopicEvent; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.SystemTopic; import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; @@ -301,7 +301,7 @@ public class BrokerService implements Closeable { private final int maxUnackedMessages; public final int maxUnackedMsgsPerDispatcher; private final AtomicBoolean blockedDispatcherOnHighUnackedMsgs = new AtomicBoolean(false); - private final Set blockedDispatchers = ConcurrentHashMap.newKeySet(); + private final Set blockedDispatchers = ConcurrentHashMap.newKeySet(); private final ReadWriteLock lock = new ReentrantReadWriteLock(); @VisibleForTesting private final DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory; @@ -3328,7 +3328,7 @@ public OrderedExecutor getTopicOrderedExecutor() { * @param dispatcher * @param numberOfMessages */ - public void addUnAckedMessages(PersistentDispatcherMultipleConsumers dispatcher, int numberOfMessages) { + public void addUnAckedMessages(AbstractPersistentDispatcherMultipleConsumers dispatcher, int numberOfMessages) { // don't block dispatchers if maxUnackedMessages = 0 if (maxUnackedMessages > 0) { totalUnackedMessages.add(numberOfMessages); @@ -3387,10 +3387,10 @@ private void blockDispatchersWithLargeUnAckMessages() { try { forEachTopic(topic -> { topic.getSubscriptions().forEach((subName, persistentSubscription) -> { - if (persistentSubscription.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentSubscription - .getDispatcher(); + if (persistentSubscription.getDispatcher() + instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentSubscription.getDispatcher(); int dispatcherUnAckMsgs = dispatcher.getTotalUnackedMessages(); if (dispatcherUnAckMsgs > maxUnackedMsgsPerDispatcher) { log.info("[{}] Blocking dispatcher due to reached max broker limit {}", @@ -3411,7 +3411,7 @@ private void blockDispatchersWithLargeUnAckMessages() { * * @param dispatcherList */ - public void unblockDispatchersOnUnAckMessages(List dispatcherList) { + public void unblockDispatchersOnUnAckMessages(List dispatcherList) { lock.writeLock().lock(); try { dispatcherList.forEach(dispatcher -> { 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 c9584f2c1790f..d25ebd0839df1 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 @@ -146,7 +146,7 @@ public class Consumer { private static final double avgPercent = 0.9; private boolean preciseDispatcherFlowControl; - private Position lastSentPositionWhenJoining; + private Position readPositionWhenJoining; private final String clientAddress; // IP address only, no port number included private final MessageId startMessageId; private final boolean isAcknowledgmentAtBatchIndexLevelEnabled; @@ -973,8 +973,8 @@ public ConsumerStatsImpl getStats() { stats.unackedMessages = unackedMessages; stats.blockedConsumerOnUnackedMsgs = blockedConsumerOnUnackedMsgs; stats.avgMessagesPerEntry = getAvgMessagesPerEntry(); - if (lastSentPositionWhenJoining != null) { - stats.lastSentPositionWhenJoining = lastSentPositionWhenJoining.toString(); + if (readPositionWhenJoining != null) { + stats.readPositionWhenJoining = readPositionWhenJoining.toString(); } return stats; } @@ -1189,8 +1189,8 @@ public boolean isPreciseDispatcherFlowControl() { return preciseDispatcherFlowControl; } - public void setLastSentPositionWhenJoining(Position lastSentPositionWhenJoining) { - this.lastSentPositionWhenJoining = lastSentPositionWhenJoining; + public void setReadPositionWhenJoining(Position readPositionWhenJoining) { + this.readPositionWhenJoining = readPositionWhenJoining; } public int getMaxUnackedMessages() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java index d1d44709a9c52..f68a9a0986b84 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java @@ -153,8 +153,8 @@ default void afterAckMessages(Throwable exOfDeletion, Object ctxOfDeletion){} /** * Trigger a new "readMoreEntries" if the dispatching has been paused before. This method is only implemented in - * {@link org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers} right now, other - * implements are not necessary to implement this method. + * {@link org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers} right now, + * other implementations do not necessary implement this method. * @return did a resume. */ default boolean checkAndResumeIfPaused(){ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java new file mode 100644 index 0000000000000..79ce308158422 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.bookkeeper.mledger.Position; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.api.proto.KeySharedMode; + +public interface StickyKeyDispatcher extends Dispatcher { + + boolean hasSameKeySharedPolicy(KeySharedMeta ksm); + + Map> getConsumerKeyHashRanges(); + + boolean isAllowOutOfOrderDelivery(); + + KeySharedMode getKeySharedMode(); + + StickyKeyConsumerSelector getSelector(); + + long getNumberOfMessagesInReplay(); + + default LinkedHashMap getRecentlyJoinedConsumers() { + return null; + } + + boolean isClassic(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java new file mode 100644 index 0000000000000..79d365b9fee21 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import java.util.Map; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; + +public abstract class AbstractPersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers + implements Dispatcher, AsyncCallbacks.ReadEntriesCallback { + public AbstractPersistentDispatcherMultipleConsumers(Subscription subscription, + ServiceConfiguration serviceConfig) { + super(subscription, serviceConfig); + } + + public abstract void unBlockDispatcherOnUnackedMsgs(); + + public abstract void readMoreEntriesAsync(); + + public abstract String getName(); + + public abstract boolean isBlockedDispatcherOnUnackedMsgs(); + + public abstract int getTotalUnackedMessages(); + + public abstract void blockDispatcherOnUnackedMsgs(); + + public abstract long getNumberOfMessagesInReplay(); + + public abstract boolean isHavePendingRead(); + + public abstract boolean isHavePendingReplayRead(); + + public abstract ManagedCursor getCursor(); + + public abstract Topic getTopic(); + + public abstract Subscription getSubscription(); + + public abstract long getDelayedTrackerMemoryUsage(); + + public abstract Map getBucketDelayedIndexStats(); + + public abstract boolean isClassic(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index b34a0b454385f..46f1f0a535650 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -41,12 +41,18 @@ public class MessageRedeliveryController { private final boolean allowOutOfOrderDelivery; + private final boolean isClassicDispatcher; private final ConcurrentBitmapSortedLongPairSet messagesToRedeliver; private final ConcurrentLongLongPairHashMap hashesToBeBlocked; private final ConcurrentLongLongHashMap hashesRefCount; public MessageRedeliveryController(boolean allowOutOfOrderDelivery) { + this(allowOutOfOrderDelivery, false); + } + + public MessageRedeliveryController(boolean allowOutOfOrderDelivery, boolean isClassicDispatcher) { this.allowOutOfOrderDelivery = allowOutOfOrderDelivery; + this.isClassicDispatcher = isClassicDispatcher; this.messagesToRedeliver = new ConcurrentBitmapSortedLongPairSet(); if (!allowOutOfOrderDelivery) { this.hashesToBeBlocked = ConcurrentLongLongPairHashMap @@ -65,7 +71,7 @@ public void add(long ledgerId, long entryId) { public void add(long ledgerId, long entryId, long stickyKeyHash) { if (!allowOutOfOrderDelivery) { - if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + if (!isClassicDispatcher && stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { throw new IllegalArgumentException("Sticky key hash is not set. It is required."); } boolean inserted = hashesToBeBlocked.putIfAbsent(ledgerId, entryId, stickyKeyHash, 0); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 73d152bab1a60..b1cd186c31784 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -39,7 +39,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.function.Predicate; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; @@ -60,7 +59,6 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.Consumer; -import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryAndMetadata; import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; @@ -85,8 +83,7 @@ /** * */ -public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers - implements Dispatcher, ReadEntriesCallback { +public class PersistentDispatcherMultipleConsumers extends AbstractPersistentDispatcherMultipleConsumers { protected final PersistentTopic topic; protected final ManagedCursor cursor; protected volatile Range lastIndividualDeletedRangeFromCursorRecovery; @@ -162,7 +159,7 @@ public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCurso this.name = topic.getName() + " / " + Codec.decode(cursor.getName()); this.topic = topic; this.dispatchMessagesThread = topic.getBrokerService().getTopicOrderedExecutor().chooseThread(); - this.redeliveryMessages = new MessageRedeliveryController(allowOutOfOrderDelivery); + this.redeliveryMessages = new MessageRedeliveryController(allowOutOfOrderDelivery, false); this.redeliveryTracker = this.serviceConfig.isSubscriptionRedeliveryTrackerEnabled() ? new InMemoryRedeliveryTracker() : RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; @@ -320,6 +317,7 @@ private synchronized void internalConsumerFlow(Consumer consumer, int additional * We should not call readMoreEntries() recursively in the same thread as there is a risk of StackOverflowError. * */ + @Override public void readMoreEntriesAsync() { // deduplication for readMoreEntriesAsync calls if (readMoreEntriesAsyncRequested.compareAndSet(false, true)) { @@ -1285,6 +1283,7 @@ public void blockDispatcherOnUnackedMsgs() { blockedDispatcherOnUnackedMsgs = TRUE; } + @Override public void unBlockDispatcherOnUnackedMsgs() { blockedDispatcherOnUnackedMsgs = FALSE; } @@ -1293,6 +1292,7 @@ public int getTotalUnackedMessages() { return totalUnackedMessages; } + @Override public String getName() { return name; } @@ -1488,6 +1488,11 @@ public Map getBucketDelayedIndexStats() { return Collections.emptyMap(); } + @Override + public boolean isClassic() { + return false; + } + public ManagedCursor getCursor() { return cursor; } @@ -1505,5 +1510,15 @@ public long getNumberOfMessagesInReplay() { return redeliveryMessages.size(); } + @Override + public boolean isHavePendingRead() { + return havePendingRead; + } + + @Override + public boolean isHavePendingReplayRead() { + return havePendingReplayRead; + } + private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumers.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java new file mode 100644 index 0000000000000..6ab7acfa56da8 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java @@ -0,0 +1,1374 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.function.Predicate; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException; +import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; +import org.apache.pulsar.broker.delayed.DelayedDeliveryTracker; +import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; +import org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker; +import org.apache.pulsar.broker.delayed.bucket.BucketDelayedDeliveryTracker; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; +import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryAndMetadata; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; +import org.apache.pulsar.broker.service.EntryBatchSizes; +import org.apache.pulsar.broker.service.InMemoryRedeliveryTracker; +import org.apache.pulsar.broker.service.RedeliveryTracker; +import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; +import org.apache.pulsar.broker.service.SendMessageInfo; +import org.apache.pulsar.broker.service.SharedConsumerAssignor; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; +import org.apache.pulsar.broker.transaction.exception.buffer.TransactionBufferException; +import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.Codec; +import org.apache.pulsar.common.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the "classic" dispatcher implementation for the Shared subscription that was used before + * Pulsar 4.0.0 and PIP-379. When `subscriptionSharedUseClassicPersistentImplementation=true`, + * this dispatcher will be used. The main purpose is to provide a way to rollback to the old behavior + * in case of issues with the preferred implementation. + */ +public class PersistentDispatcherMultipleConsumersClassic extends AbstractPersistentDispatcherMultipleConsumers { + protected final PersistentTopic topic; + protected final ManagedCursor cursor; + protected volatile Range lastIndividualDeletedRangeFromCursorRecovery; + + private CompletableFuture closeFuture = null; + protected final MessageRedeliveryController redeliveryMessages; + protected final RedeliveryTracker redeliveryTracker; + + private Optional delayedDeliveryTracker = Optional.empty(); + + protected volatile boolean havePendingRead = false; + protected volatile boolean havePendingReplayRead = false; + protected volatile Position minReplayedPosition = null; + protected boolean shouldRewindBeforeReadingOrReplaying = false; + protected final String name; + private boolean sendInProgress = false; + protected static final AtomicIntegerFieldUpdater + TOTAL_AVAILABLE_PERMITS_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "totalAvailablePermits"); + protected volatile int totalAvailablePermits = 0; + protected volatile int readBatchSize; + protected final Backoff readFailureBackoff; + private static final AtomicIntegerFieldUpdater + TOTAL_UNACKED_MESSAGES_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "totalUnackedMessages"); + protected volatile int totalUnackedMessages = 0; + /** + * A signature that relate to the check of "Dispatching has paused on cursor data can fully persist". + * Note: It is a tool that helps determine whether it should trigger a new reading after acknowledgments to avoid + * too many CPU circles, see {@link #afterAckMessages(Throwable, Object)} for more details. Do not use this + * to confirm whether the delivery should be paused, please call {@link #shouldPauseOnAckStatePersist}. + */ + protected static final AtomicIntegerFieldUpdater + BLOCKED_DISPATCHER_ON_CURSOR_DATA_CAN_NOT_FULLY_PERSIST_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "blockedDispatcherOnCursorDataCanNotFullyPersist"); + private volatile int blockedDispatcherOnCursorDataCanNotFullyPersist = FALSE; + private volatile int blockedDispatcherOnUnackedMsgs = FALSE; + protected static final AtomicIntegerFieldUpdater + BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "blockedDispatcherOnUnackedMsgs"); + protected Optional dispatchRateLimiter = Optional.empty(); + private AtomicBoolean isRescheduleReadInProgress = new AtomicBoolean(false); + protected final ExecutorService dispatchMessagesThread; + private final SharedConsumerAssignor assignor; + + protected enum ReadType { + Normal, Replay + } + + public PersistentDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, + Subscription subscription) { + this(topic, cursor, subscription, true); + } + + public PersistentDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, + Subscription subscription, + boolean allowOutOfOrderDelivery) { + super(subscription, topic.getBrokerService().pulsar().getConfiguration()); + this.cursor = cursor; + this.lastIndividualDeletedRangeFromCursorRecovery = cursor.getLastIndividualDeletedRange(); + this.name = topic.getName() + " / " + Codec.decode(cursor.getName()); + this.topic = topic; + this.dispatchMessagesThread = topic.getBrokerService().getTopicOrderedExecutor().chooseThread(); + this.redeliveryMessages = new MessageRedeliveryController(allowOutOfOrderDelivery, true); + this.redeliveryTracker = this.serviceConfig.isSubscriptionRedeliveryTrackerEnabled() + ? new InMemoryRedeliveryTracker() + : RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; + this.readBatchSize = serviceConfig.getDispatcherMaxReadBatchSize(); + this.initializeDispatchRateLimiterIfNeeded(); + this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addMessageToReplay); + this.readFailureBackoff = new Backoff( + topic.getBrokerService().pulsar().getConfiguration().getDispatcherReadFailureBackoffInitialTimeInMs(), + TimeUnit.MILLISECONDS, + 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + } + + @Override + public synchronized CompletableFuture addConsumer(Consumer consumer) { + if (IS_CLOSED_UPDATER.get(this) == TRUE) { + log.warn("[{}] Dispatcher is already closed. Closing consumer {}", name, consumer); + consumer.disconnect(); + return CompletableFuture.completedFuture(null); + } + if (consumerList.isEmpty()) { + if (havePendingRead || havePendingReplayRead) { + // There is a pending read from previous run. We must wait for it to complete and then rewind + shouldRewindBeforeReadingOrReplaying = true; + } else { + cursor.rewind(); + shouldRewindBeforeReadingOrReplaying = false; + } + redeliveryMessages.clear(); + delayedDeliveryTracker.ifPresent(tracker -> { + // Don't clean up BucketDelayedDeliveryTracker, otherwise we will lose the bucket snapshot + if (tracker instanceof InMemoryDelayedDeliveryTracker) { + tracker.clear(); + } + }); + } + + if (isConsumersExceededOnSubscription()) { + log.warn("[{}] Attempting to add consumer to subscription which reached max consumers limit {}", + name, consumer); + return FutureUtil.failedFuture(new ConsumerBusyException("Subscription reached max consumers limit")); + } + // This is not an expected scenario, it will never happen in expected. Just print a warn log if the unexpected + // scenario happens. See more detail: https://github.com/apache/pulsar/pull/22283. + if (consumerSet.contains(consumer)) { + log.warn("[{}] Attempting to add a consumer that already registered {}", name, consumer); + } + + consumerList.add(consumer); + if (consumerList.size() > 1 + && consumer.getPriorityLevel() < consumerList.get(consumerList.size() - 2).getPriorityLevel()) { + consumerList.sort(Comparator.comparingInt(Consumer::getPriorityLevel)); + } + consumerSet.add(consumer); + + return CompletableFuture.completedFuture(null); + } + + @Override + protected boolean isConsumersExceededOnSubscription() { + return isConsumersExceededOnSubscription(topic, consumerList.size()); + } + + @Override + public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { + // decrement unack-message count for removed consumer + addUnAckedMessages(-consumer.getUnackedMessages()); + if (consumerSet.removeAll(consumer) == 1) { + consumerList.remove(consumer); + log.info("Removed consumer {} with pending {} acks", consumer, consumer.getPendingAcks().size()); + if (consumerList.isEmpty()) { + clearComponentsAfterRemovedAllConsumers(); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Consumer are left, reading more entries", name); + } + consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + }); + totalAvailablePermits -= consumer.getAvailablePermits(); + if (log.isDebugEnabled()) { + log.debug("[{}] Decreased totalAvailablePermits by {} in PersistentDispatcherMultipleConsumers. " + + "New dispatcher permit count is {}", name, consumer.getAvailablePermits(), + totalAvailablePermits); + } + readMoreEntries(); + } + } else { + /** + * This is not an expected scenario, it will never happen in expected. + * Just add a defensive code to avoid the topic can not be unloaded anymore: remove the consumers which + * are not mismatch with {@link #consumerSet}. See more detail: https://github.com/apache/pulsar/pull/22270. + */ + log.error("[{}] Trying to remove a non-connected consumer: {}", name, consumer); + consumerList.removeIf(c -> consumer.equals(c)); + if (consumerList.isEmpty()) { + clearComponentsAfterRemovedAllConsumers(); + } + } + } + + private synchronized void clearComponentsAfterRemovedAllConsumers() { + cancelPendingRead(); + + redeliveryMessages.clear(); + redeliveryTracker.clear(); + if (closeFuture != null) { + log.info("[{}] All consumers removed. Subscription is disconnected", name); + closeFuture.complete(null); + } + totalAvailablePermits = 0; + } + + @Override + public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) { + topic.getBrokerService().executor().execute(() -> { + internalConsumerFlow(consumer, additionalNumberOfMessages); + }); + } + + private synchronized void internalConsumerFlow(Consumer consumer, int additionalNumberOfMessages) { + if (!consumerSet.contains(consumer)) { + if (log.isDebugEnabled()) { + log.debug("[{}] Ignoring flow control from disconnected consumer {}", name, consumer); + } + return; + } + + totalAvailablePermits += additionalNumberOfMessages; + + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Trigger new read after receiving flow control message with permits {} " + + "after adding {} permits", name, consumer, + totalAvailablePermits, additionalNumberOfMessages); + } + readMoreEntries(); + } + + /** + * We should not call readMoreEntries() recursively in the same thread as there is a risk of StackOverflowError. + * + */ + public void readMoreEntriesAsync() { + topic.getBrokerService().executor().execute(this::readMoreEntries); + } + + public synchronized void readMoreEntries() { + if (cursor.isClosed()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor is already closed, skipping read more entries.", cursor.getName()); + } + return; + } + if (isSendInProgress()) { + // we cannot read more entries while sending the previous batch + // otherwise we could re-read the same entries and send duplicates + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic, Due to sending in-progress.", + topic.getName(), getSubscriptionName()); + } + return; + } + if (shouldPauseDeliveryForDelayTracker()) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic, Due to pause delivery for delay tracker.", + topic.getName(), getSubscriptionName()); + } + return; + } + if (topic.isTransferring()) { + // Do not deliver messages for topics that are undergoing transfer, as the acknowledgments would be ignored. + return; + } + + // totalAvailablePermits may be updated by other threads + int firstAvailableConsumerPermits = getFirstAvailableConsumerPermits(); + int currentTotalAvailablePermits = Math.max(totalAvailablePermits, firstAvailableConsumerPermits); + if (currentTotalAvailablePermits > 0 && firstAvailableConsumerPermits > 0) { + Pair calculateResult = calculateToRead(currentTotalAvailablePermits); + int messagesToRead = calculateResult.getLeft(); + long bytesToRead = calculateResult.getRight(); + + if (messagesToRead == -1 || bytesToRead == -1) { + // Skip read as topic/dispatcher has exceed the dispatch rate or previous pending read hasn't complete. + return; + } + + NavigableSet messagesToReplayNow = getMessagesToReplayNow(messagesToRead); + NavigableSet messagesToReplayFiltered = filterOutEntriesWillBeDiscarded(messagesToReplayNow); + if (!messagesToReplayFiltered.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Schedule replay of {} messages for {} consumers", name, + messagesToReplayFiltered.size(), consumerList.size()); + } + + havePendingReplayRead = true; + minReplayedPosition = messagesToReplayNow.first(); + Set deletedMessages = topic.isDelayedDeliveryEnabled() + ? asyncReplayEntriesInOrder(messagesToReplayFiltered) + : asyncReplayEntries(messagesToReplayFiltered); + // clear already acked positions from replay bucket + + deletedMessages.forEach(position -> redeliveryMessages.remove(((Position) position).getLedgerId(), + ((Position) position).getEntryId())); + // if all the entries are acked-entries and cleared up from redeliveryMessages, try to read + // next entries as readCompletedEntries-callback was never called + if ((messagesToReplayFiltered.size() - deletedMessages.size()) == 0) { + havePendingReplayRead = false; + readMoreEntriesAsync(); + } + } else if (BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.get(this) == TRUE) { + if (log.isDebugEnabled()) { + log.debug("[{}] Dispatcher read is blocked due to unackMessages {} reached to max {}", name, + totalUnackedMessages, topic.getMaxUnackedMessagesOnSubscription()); + } + } else if (!havePendingRead && hasConsumersNeededNormalRead()) { + if (shouldPauseOnAckStatePersist(ReadType.Normal)) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic, Due to blocked on ack state persistent.", + topic.getName(), getSubscriptionName()); + } + return; + } + if (log.isDebugEnabled()) { + log.debug("[{}] Schedule read of {} messages for {} consumers", name, messagesToRead, + consumerList.size()); + } + havePendingRead = true; + NavigableSet toReplay = getMessagesToReplayNow(1); + if (!toReplay.isEmpty()) { + minReplayedPosition = toReplay.first(); + redeliveryMessages.add(minReplayedPosition.getLedgerId(), minReplayedPosition.getEntryId()); + } else { + minReplayedPosition = null; + } + + // Filter out and skip read delayed messages exist in DelayedDeliveryTracker + if (delayedDeliveryTracker.isPresent()) { + Predicate skipCondition = null; + final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get(); + if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { + skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker) + .containsMessage(position.getLedgerId(), position.getEntryId()); + } + cursor.asyncReadEntriesWithSkipOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, + topic.getMaxReadPosition(), skipCondition); + } else { + cursor.asyncReadEntriesOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, + topic.getMaxReadPosition()); + } + } else { + if (log.isDebugEnabled()) { + if (!messagesToReplayNow.isEmpty()) { + log.debug("[{}] [{}] Skipping read for the topic: because all entries in replay queue were" + + " filtered out due to the mechanism of Key_Shared mode, and the left consumers have" + + " no permits now", + topic.getName(), getSubscriptionName()); + } else { + log.debug("[{}] Cannot schedule next read until previous one is done", name); + } + } + } + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Consumer buffer is full, pause reading", name); + } + } + } + + private boolean shouldPauseOnAckStatePersist(ReadType readType) { + // Allows new consumers to consume redelivered messages caused by the just-closed consumer. + if (readType != ReadType.Normal) { + return false; + } + if (!((PersistentTopic) subscription.getTopic()).isDispatcherPauseOnAckStatePersistentEnabled()) { + return false; + } + if (cursor == null) { + return true; + } + return blockedDispatcherOnCursorDataCanNotFullyPersist == TRUE; + } + + @Override + protected void reScheduleRead() { + if (isRescheduleReadInProgress.compareAndSet(false, true)) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Reschedule message read in {} ms", topic.getName(), name, MESSAGE_RATE_BACKOFF_MS); + } + topic.getBrokerService().executor().schedule( + () -> { + isRescheduleReadInProgress.set(false); + readMoreEntries(); + }, + MESSAGE_RATE_BACKOFF_MS, TimeUnit.MILLISECONDS); + } + } + + // left pair is messagesToRead, right pair is bytesToRead + protected Pair calculateToRead(int currentTotalAvailablePermits) { + int messagesToRead = Math.min(currentTotalAvailablePermits, readBatchSize); + long bytesToRead = serviceConfig.getDispatcherMaxReadSizeBytes(); + + Consumer c = getRandomConsumer(); + // if turn on precise dispatcher flow control, adjust the record to read + if (c != null && c.isPreciseDispatcherFlowControl()) { + int avgMessagesPerEntry = Math.max(1, c.getAvgMessagesPerEntry()); + messagesToRead = Math.min( + (int) Math.ceil(currentTotalAvailablePermits * 1.0 / avgMessagesPerEntry), + readBatchSize); + } + + if (!isConsumerWritable()) { + // If the connection is not currently writable, we issue the read request anyway, but for a single + // message. The intent here is to keep use the request as a notification mechanism while avoiding to + // read and dispatch a big batch of messages which will need to wait before getting written to the + // socket. + messagesToRead = 1; + } + + // throttle only if: (1) cursor is not active (or flag for throttle-nonBacklogConsumer is enabled) bcz + // active-cursor reads message from cache rather from bookkeeper (2) if topic has reached message-rate + // threshold: then schedule the read after MESSAGE_RATE_BACKOFF_MS + if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { + if (topic.getBrokerDispatchRateLimiter().isPresent()) { + DispatchRateLimiter brokerRateLimiter = topic.getBrokerDispatchRateLimiter().get(); + Pair calculateToRead = + updateMessagesToRead(brokerRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] message-read exceeded broker message-rate {}/{}, schedule after a {}", name, + brokerRateLimiter.getDispatchRateOnMsg(), brokerRateLimiter.getDispatchRateOnByte(), + MESSAGE_RATE_BACKOFF_MS); + } + reScheduleRead(); + return Pair.of(-1, -1L); + } + } + + if (topic.getDispatchRateLimiter().isPresent()) { + DispatchRateLimiter topicRateLimiter = topic.getDispatchRateLimiter().get(); + Pair calculateToRead = + updateMessagesToRead(topicRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] message-read exceeded topic message-rate {}/{}, schedule after a {}", name, + topicRateLimiter.getDispatchRateOnMsg(), topicRateLimiter.getDispatchRateOnByte(), + MESSAGE_RATE_BACKOFF_MS); + } + reScheduleRead(); + return Pair.of(-1, -1L); + } + } + + if (dispatchRateLimiter.isPresent()) { + Pair calculateToRead = + updateMessagesToRead(dispatchRateLimiter.get(), messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] message-read exceeded subscription message-rate {}/{}, schedule after a {}", + name, dispatchRateLimiter.get().getDispatchRateOnMsg(), + dispatchRateLimiter.get().getDispatchRateOnByte(), + MESSAGE_RATE_BACKOFF_MS); + } + reScheduleRead(); + return Pair.of(-1, -1L); + } + } + } + + if (havePendingReplayRead) { + if (log.isDebugEnabled()) { + log.debug("[{}] Skipping replay while awaiting previous read to complete", name); + } + return Pair.of(-1, -1L); + } + + // If messagesToRead is 0 or less, correct it to 1 to prevent IllegalArgumentException + messagesToRead = Math.max(messagesToRead, 1); + bytesToRead = Math.max(bytesToRead, 1); + return Pair.of(messagesToRead, bytesToRead); + } + + protected Set asyncReplayEntries(Set positions) { + return cursor.asyncReplayEntries(positions, this, ReadType.Replay); + } + + protected Set asyncReplayEntriesInOrder(Set positions) { + return cursor.asyncReplayEntries(positions, this, ReadType.Replay, true); + } + + @Override + public boolean isConsumerConnected() { + return !consumerList.isEmpty(); + } + + @Override + public CopyOnWriteArrayList getConsumers() { + return consumerList; + } + + @Override + public synchronized boolean canUnsubscribe(Consumer consumer) { + return consumerList.size() == 1 && consumerSet.contains(consumer); + } + + @Override + public CompletableFuture close(boolean disconnectConsumers, + Optional assignedBrokerLookupData) { + IS_CLOSED_UPDATER.set(this, TRUE); + + Optional delayedDeliveryTracker; + synchronized (this) { + delayedDeliveryTracker = this.delayedDeliveryTracker; + this.delayedDeliveryTracker = Optional.empty(); + } + + delayedDeliveryTracker.ifPresent(DelayedDeliveryTracker::close); + dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); + + return disconnectConsumers + ? disconnectAllConsumers(false, assignedBrokerLookupData) : CompletableFuture.completedFuture(null); + } + + @Override + public synchronized CompletableFuture disconnectAllConsumers( + boolean isResetCursor, Optional assignedBrokerLookupData) { + closeFuture = new CompletableFuture<>(); + if (consumerList.isEmpty()) { + closeFuture.complete(null); + } else { + // Iterator of CopyOnWriteArrayList uses the internal array to do the for-each, and CopyOnWriteArrayList + // will create a new internal array when adding/removing a new item. So remove items in the for-each + // block is safety when the for-each and add/remove are using a same lock. + consumerList.forEach(consumer -> consumer.disconnect(isResetCursor, assignedBrokerLookupData)); + cancelPendingRead(); + } + return closeFuture; + } + + @Override + protected void cancelPendingRead() { + if ((havePendingRead || havePendingReplayRead) && cursor.cancelPendingReadRequest()) { + havePendingRead = false; + havePendingReplayRead = false; + } + } + + @Override + public CompletableFuture disconnectActiveConsumers(boolean isResetCursor) { + return disconnectAllConsumers(isResetCursor); + } + + @Override + public synchronized void resetCloseFuture() { + closeFuture = null; + } + + @Override + public void reset() { + resetCloseFuture(); + IS_CLOSED_UPDATER.set(this, FALSE); + } + + @Override + public SubType getType() { + return SubType.Shared; + } + + @Override + public final synchronized void readEntriesComplete(List entries, Object ctx) { + ReadType readType = (ReadType) ctx; + if (readType == ReadType.Normal) { + havePendingRead = false; + } else { + havePendingReplayRead = false; + } + + if (readBatchSize < serviceConfig.getDispatcherMaxReadBatchSize()) { + int newReadBatchSize = Math.min(readBatchSize * 2, serviceConfig.getDispatcherMaxReadBatchSize()); + if (log.isDebugEnabled()) { + log.debug("[{}] Increasing read batch size from {} to {}", name, readBatchSize, newReadBatchSize); + } + + readBatchSize = newReadBatchSize; + } + + readFailureBackoff.reduceToHalf(); + + if (shouldRewindBeforeReadingOrReplaying && readType == ReadType.Normal) { + // All consumers got disconnected before the completion of the read operation + entries.forEach(Entry::release); + cursor.rewind(); + shouldRewindBeforeReadingOrReplaying = false; + readMoreEntries(); + return; + } + + if (log.isDebugEnabled()) { + log.debug("[{}] Distributing {} messages to {} consumers", name, entries.size(), consumerList.size()); + } + + long size = entries.stream().mapToLong(Entry::getLength).sum(); + updatePendingBytesToDispatch(size); + + // dispatch messages to a separate thread, but still in order for this subscription + // sendMessagesToConsumers is responsible for running broker-side filters + // that may be quite expensive + if (serviceConfig.isDispatcherDispatchMessagesInSubscriptionThread()) { + // setting sendInProgress here, because sendMessagesToConsumers will be executed + // in a separate thread, and we want to prevent more reads + acquireSendInProgress(); + dispatchMessagesThread.execute(() -> { + if (sendMessagesToConsumers(readType, entries, false)) { + updatePendingBytesToDispatch(-size); + readMoreEntries(); + } else { + updatePendingBytesToDispatch(-size); + } + }); + } else { + if (sendMessagesToConsumers(readType, entries, true)) { + updatePendingBytesToDispatch(-size); + readMoreEntriesAsync(); + } else { + updatePendingBytesToDispatch(-size); + } + } + } + + protected synchronized void acquireSendInProgress() { + sendInProgress = true; + } + + protected synchronized void releaseSendInProgress() { + sendInProgress = false; + } + + protected synchronized boolean isSendInProgress() { + return sendInProgress; + } + + protected final synchronized boolean sendMessagesToConsumers(ReadType readType, List entries, + boolean needAcquireSendInProgress) { + if (needAcquireSendInProgress) { + acquireSendInProgress(); + } + try { + return trySendMessagesToConsumers(readType, entries); + } finally { + releaseSendInProgress(); + } + } + + /** + * Dispatch the messages to the Consumers. + * @return true if you want to trigger a new read. + * This method is overridden by other classes, please take a look to other implementations + * if you need to change it. + */ + protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { + if (needTrimAckedMessages()) { + cursor.trimDeletedEntries(entries); + } + + int entriesToDispatch = entries.size(); + // Trigger read more messages + if (entriesToDispatch == 0) { + return true; + } + final MessageMetadata[] metadataArray = new MessageMetadata[entries.size()]; + int remainingMessages = 0; + boolean hasChunk = false; + for (int i = 0; i < metadataArray.length; i++) { + final MessageMetadata metadata = Commands.peekAndCopyMessageMetadata( + entries.get(i).getDataBuffer(), subscription.toString(), -1); + if (metadata != null) { + remainingMessages += metadata.getNumMessagesInBatch(); + if (!hasChunk && metadata.hasUuid()) { + hasChunk = true; + } + } + metadataArray[i] = metadata; + } + if (hasChunk) { + return sendChunkedMessagesToConsumers(readType, entries, metadataArray); + } + + int start = 0; + long totalMessagesSent = 0; + long totalBytesSent = 0; + long totalEntries = 0; + int avgBatchSizePerMsg = remainingMessages > 0 ? Math.max(remainingMessages / entries.size(), 1) : 1; + + // If the dispatcher is closed, firstAvailableConsumerPermits will be 0, which skips dispatching the + // messages. + while (entriesToDispatch > 0 && isAtleastOneConsumerAvailable()) { + Consumer c = getNextConsumer(); + if (c == null) { + // Do nothing, cursor will be rewind at reconnection + log.info("[{}] rewind because no available consumer found from total {}", name, consumerList.size()); + entries.subList(start, entries.size()).forEach(Entry::release); + cursor.rewind(); + return false; + } + + // round-robin dispatch batch size for this consumer + int availablePermits = c.isWritable() ? c.getAvailablePermits() : 1; + if (c.getMaxUnackedMessages() > 0) { + // Avoid negative number + int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + availablePermits = Math.min(availablePermits, remainUnAckedMessages); + } + if (log.isDebugEnabled() && !c.isWritable()) { + log.debug("[{}-{}] consumer is not writable. dispatching only 1 message to {}; " + + "availablePermits are {}", topic.getName(), name, + c, c.getAvailablePermits()); + } + + int messagesForC = Math.min(Math.min(remainingMessages, availablePermits), + serviceConfig.getDispatcherMaxRoundRobinBatchSize()); + messagesForC = Math.max(messagesForC / avgBatchSizePerMsg, 1); + + int end = Math.min(start + messagesForC, entries.size()); + List entriesForThisConsumer = entries.subList(start, end); + + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { + entriesForThisConsumer.forEach(entry -> { + redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); + }); + } + + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + + EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesForThisConsumer.size()); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(entriesForThisConsumer.size()); + totalEntries += filterEntriesForConsumer(metadataArray, start, + entriesForThisConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, + readType == ReadType.Replay, c); + + c.sendMessages(entriesForThisConsumer, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), + sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), redeliveryTracker); + + int msgSent = sendMessageInfo.getTotalMessages(); + remainingMessages -= msgSent; + start += messagesForC; + entriesToDispatch -= messagesForC; + TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, + -(msgSent - batchIndexesAcks.getTotalAckedIndexCount())); + if (log.isDebugEnabled()) { + log.debug("[{}] Added -({} minus {}) permits to TOTAL_AVAILABLE_PERMITS_UPDATER in " + + "PersistentDispatcherMultipleConsumers", + name, msgSent, batchIndexesAcks.getTotalAckedIndexCount()); + } + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); + } + + acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); + + if (entriesToDispatch > 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] No consumers found with available permits, storing {} positions for later replay", name, + entries.size() - start); + } + entries.subList(start, entries.size()).forEach(entry -> { + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + }); + } + return true; + } + + private boolean sendChunkedMessagesToConsumers(ReadType readType, + List entries, + MessageMetadata[] metadataArray) { + final List originalEntryAndMetadataList = new ArrayList<>(metadataArray.length); + for (int i = 0; i < metadataArray.length; i++) { + originalEntryAndMetadataList.add(EntryAndMetadata.create(entries.get(i), metadataArray[i])); + } + + final Map> assignResult = + assignor.assign(originalEntryAndMetadataList, consumerList.size()); + long totalMessagesSent = 0; + long totalBytesSent = 0; + long totalEntries = 0; + final AtomicInteger numConsumers = new AtomicInteger(assignResult.size()); + for (Map.Entry> current : assignResult.entrySet()) { + final Consumer consumer = current.getKey(); + final List entryAndMetadataList = current.getValue(); + final int messagesForC = Math.min(consumer.getAvailablePermits(), entryAndMetadataList.size()); + if (log.isDebugEnabled()) { + log.debug("[{}] select consumer {} with messages num {}, read type is {}", + name, consumer.consumerName(), messagesForC, readType); + } + if (messagesForC < entryAndMetadataList.size()) { + for (int i = messagesForC; i < entryAndMetadataList.size(); i++) { + final EntryAndMetadata entry = entryAndMetadataList.get(i); + addMessageToReplay(entry); + entryAndMetadataList.set(i, null); + } + } + if (messagesForC == 0) { + numConsumers.decrementAndGet(); + continue; + } + if (readType == ReadType.Replay) { + entryAndMetadataList.stream().limit(messagesForC) + .forEach(e -> redeliveryMessages.remove(e.getLedgerId(), e.getEntryId())); + } + final SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + final EntryBatchSizes batchSizes = EntryBatchSizes.get(messagesForC); + final EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); + + totalEntries += filterEntriesForConsumer(entryAndMetadataList, batchSizes, sendMessageInfo, + batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + consumer.sendMessages(entryAndMetadataList, batchSizes, batchIndexesAcks, + sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), + sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker() + ).addListener(future -> { + if (future.isDone() && numConsumers.decrementAndGet() == 0) { + readMoreEntries(); + } + }); + + TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, + -(sendMessageInfo.getTotalMessages() - batchIndexesAcks.getTotalAckedIndexCount())); + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); + } + + acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); + + return numConsumers.get() == 0; // trigger a new readMoreEntries() call + } + + @Override + public synchronized void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + + ReadType readType = (ReadType) ctx; + long waitTimeMillis = readFailureBackoff.next(); + + // Do not keep reading more entries if the cursor is already closed. + if (exception instanceof ManagedLedgerException.CursorAlreadyClosedException) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor is already closed, skipping read more entries", cursor.getName()); + } + // Set the wait time to -1 to avoid rescheduling the read. + waitTimeMillis = -1; + } else if (exception instanceof NoMoreEntriesToReadException) { + if (cursor.getNumberOfEntriesInBacklog(false) == 0) { + // Topic has been terminated and there are no more entries to read + // Notify the consumer only if all the messages were already acknowledged + checkAndApplyReachedEndOfTopicOrTopicMigration(consumerList); + } + } else if (exception.getCause() instanceof TransactionBufferException.TransactionNotSealedException + || exception.getCause() instanceof ManagedLedgerException.OffloadReadHandleClosedException) { + waitTimeMillis = 1; + if (log.isDebugEnabled()) { + log.debug("[{}] Error reading transaction entries : {}, Read Type {} - Retrying to read in {} seconds", + name, exception.getMessage(), readType, waitTimeMillis / 1000.0); + } + } else if (!(exception instanceof TooManyRequestsException)) { + log.error("[{}] Error reading entries at {} : {}, Read Type {} - Retrying to read in {} seconds", name, + cursor.getReadPosition(), exception.getMessage(), readType, waitTimeMillis / 1000.0); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Error reading entries at {} : {}, Read Type {} - Retrying to read in {} seconds", name, + cursor.getReadPosition(), exception.getMessage(), readType, waitTimeMillis / 1000.0); + } + } + + if (shouldRewindBeforeReadingOrReplaying) { + shouldRewindBeforeReadingOrReplaying = false; + cursor.rewind(); + } + + if (readType == ReadType.Normal) { + havePendingRead = false; + } else { + havePendingReplayRead = false; + if (exception instanceof ManagedLedgerException.InvalidReplayPositionException) { + Position markDeletePosition = (Position) cursor.getMarkDeletedPosition(); + redeliveryMessages.removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); + } + } + + readBatchSize = serviceConfig.getDispatcherMinReadBatchSize(); + // Skip read if the waitTimeMillis is a nagetive value. + if (waitTimeMillis >= 0) { + scheduleReadEntriesWithDelay(exception, readType, waitTimeMillis); + } + } + + @VisibleForTesting + void scheduleReadEntriesWithDelay(Exception e, ReadType readType, long waitTimeMillis) { + topic.getBrokerService().executor().schedule(() -> { + synchronized (PersistentDispatcherMultipleConsumersClassic.this) { + // If it's a replay read we need to retry even if there's already + // another scheduled read, otherwise we'd be stuck until + // more messages are published. + if (!havePendingRead || readType == ReadType.Replay) { + log.info("[{}] Retrying read operation", name); + readMoreEntries(); + } else { + log.info("[{}] Skipping read retry: havePendingRead {}", name, havePendingRead, e); + } + } + }, waitTimeMillis, TimeUnit.MILLISECONDS); + } + + private boolean needTrimAckedMessages() { + if (lastIndividualDeletedRangeFromCursorRecovery == null) { + return false; + } else { + return lastIndividualDeletedRangeFromCursorRecovery.upperEndpoint() + .compareTo((Position) cursor.getReadPosition()) > 0; + } + } + + /** + * returns true only if {@link AbstractDispatcherMultipleConsumers#consumerList} + * has atleast one unblocked consumer and have available permits. + * + * @return + */ + protected boolean isAtleastOneConsumerAvailable() { + return getFirstAvailableConsumerPermits() > 0; + } + + protected int getFirstAvailableConsumerPermits() { + if (consumerList.isEmpty() || IS_CLOSED_UPDATER.get(this) == TRUE) { + // abort read if no consumers are connected or if disconnect is initiated + return 0; + } + for (Consumer consumer : consumerList) { + if (consumer != null && !consumer.isBlocked()) { + int availablePermits = consumer.getAvailablePermits(); + if (availablePermits > 0) { + return availablePermits; + } + } + } + return 0; + } + + private boolean isConsumerWritable() { + for (Consumer consumer : consumerList) { + if (consumer.isWritable()) { + return true; + } + } + if (log.isDebugEnabled()) { + log.debug("[{}-{}] consumer is not writable", topic.getName(), name); + } + return false; + } + + @Override + public boolean isConsumerAvailable(Consumer consumer) { + return consumer != null && !consumer.isBlocked() && consumer.getAvailablePermits() > 0; + } + + @Override + public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch) { + consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + if (addMessageToReplay(ledgerId, entryId, stickyKeyHash)) { + redeliveryTracker.incrementAndGetRedeliveryCount((PositionFactory.create(ledgerId, entryId))); + } + }); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, + redeliveryMessages); + } + readMoreEntries(); + } + + @Override + public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + positions.forEach(position -> { + // TODO: We want to pass a sticky key hash as a third argument to guarantee the order of the messages + // on Key_Shared subscription, but it's difficult to get the sticky key here + if (addMessageToReplay(position.getLedgerId(), position.getEntryId())) { + redeliveryTracker.incrementAndGetRedeliveryCount(position); + } + }); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions); + } + readMoreEntries(); + } + + @Override + public void addUnAckedMessages(int numberOfMessages) { + int maxUnackedMessages = topic.getMaxUnackedMessagesOnSubscription(); + // don't block dispatching if maxUnackedMessages = 0 + if (maxUnackedMessages <= 0 && blockedDispatcherOnUnackedMsgs == TRUE + && BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, TRUE, FALSE)) { + log.info("[{}] Dispatcher is unblocked, since maxUnackedMessagesPerSubscription=0", name); + readMoreEntriesAsync(); + } + + int unAckedMessages = TOTAL_UNACKED_MESSAGES_UPDATER.addAndGet(this, numberOfMessages); + if (unAckedMessages >= maxUnackedMessages && maxUnackedMessages > 0 + && BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, FALSE, TRUE)) { + // block dispatcher if it reaches maxUnAckMsg limit + log.debug("[{}] Dispatcher is blocked due to unackMessages {} reached to max {}", name, + unAckedMessages, maxUnackedMessages); + } else if (topic.getBrokerService().isBrokerDispatchingBlocked() + && blockedDispatcherOnUnackedMsgs == TRUE) { + // unblock dispatcher: if dispatcher is blocked due to broker-unackMsg limit and if it ack back enough + // messages + if (totalUnackedMessages < (topic.getBrokerService().maxUnackedMsgsPerDispatcher / 2)) { + if (BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, TRUE, FALSE)) { + // it removes dispatcher from blocked list and unblocks dispatcher by scheduling read + topic.getBrokerService().unblockDispatchersOnUnAckMessages(Lists.newArrayList(this)); + } + } + } else if (blockedDispatcherOnUnackedMsgs == TRUE && unAckedMessages < maxUnackedMessages / 2) { + // unblock dispatcher if it acks back enough messages + if (BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, TRUE, FALSE)) { + log.debug("[{}] Dispatcher is unblocked", name); + readMoreEntriesAsync(); + } + } + // increment broker-level count + topic.getBrokerService().addUnAckedMessages(this, numberOfMessages); + } + + @Override + public void afterAckMessages(Throwable exOfDeletion, Object ctxOfDeletion) { + boolean unPaused = blockedDispatcherOnCursorDataCanNotFullyPersist == FALSE; + // Trigger a new read if needed. + boolean shouldPauseNow = !checkAndResumeIfPaused(); + // Switch stat to "paused" if needed. + if (unPaused && shouldPauseNow) { + if (!BLOCKED_DISPATCHER_ON_CURSOR_DATA_CAN_NOT_FULLY_PERSIST_UPDATER + .compareAndSet(this, FALSE, TRUE)) { + // Retry due to conflict update. + afterAckMessages(exOfDeletion, ctxOfDeletion); + } + } + } + + @Override + public boolean checkAndResumeIfPaused() { + boolean paused = blockedDispatcherOnCursorDataCanNotFullyPersist == TRUE; + // Calling "cursor.isCursorDataFullyPersistable()" will loop the collection "individualDeletedMessages". It is + // not a light method. + // If never enabled "dispatcherPauseOnAckStatePersistentEnabled", skip the following checks to improve + // performance. + if (!paused && !topic.isDispatcherPauseOnAckStatePersistentEnabled()){ + // "true" means no need to pause. + return true; + } + // Enabled "dispatcherPauseOnAckStatePersistentEnabled" before. + boolean shouldPauseNow = !cursor.isCursorDataFullyPersistable() + && topic.isDispatcherPauseOnAckStatePersistentEnabled(); + // No need to change. + if (paused == shouldPauseNow) { + return !shouldPauseNow; + } + // Should change to "un-pause". + if (paused && !shouldPauseNow) { + // If there was no previous pause due to cursor data is too large to persist, we don't need to manually + // trigger a new read. This can avoid too many CPU circles. + if (BLOCKED_DISPATCHER_ON_CURSOR_DATA_CAN_NOT_FULLY_PERSIST_UPDATER.compareAndSet(this, TRUE, FALSE)) { + readMoreEntriesAsync(); + } else { + // Retry due to conflict update. + checkAndResumeIfPaused(); + } + } + return !shouldPauseNow; + } + + public boolean isBlockedDispatcherOnUnackedMsgs() { + return blockedDispatcherOnUnackedMsgs == TRUE; + } + + public void blockDispatcherOnUnackedMsgs() { + blockedDispatcherOnUnackedMsgs = TRUE; + } + + public void unBlockDispatcherOnUnackedMsgs() { + blockedDispatcherOnUnackedMsgs = FALSE; + } + + public int getTotalUnackedMessages() { + return totalUnackedMessages; + } + + public String getName() { + return name; + } + + @Override + public RedeliveryTracker getRedeliveryTracker() { + return redeliveryTracker; + } + + @Override + public Optional getRateLimiter() { + return dispatchRateLimiter; + } + + @Override + public boolean initializeDispatchRateLimiterIfNeeded() { + if (!dispatchRateLimiter.isPresent() && DispatchRateLimiter.isDispatchRateEnabled( + topic.getSubscriptionDispatchRate(getSubscriptionName()))) { + this.dispatchRateLimiter = + Optional.of(new DispatchRateLimiter(topic, getSubscriptionName(), Type.SUBSCRIPTION)); + return true; + } + return false; + } + + @Override + public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata msgMetadata) { + if (!topic.isDelayedDeliveryEnabled()) { + // If broker has the feature disabled, always deliver messages immediately + return false; + } + + synchronized (this) { + if (delayedDeliveryTracker.isEmpty()) { + if (!msgMetadata.hasDeliverAtTime()) { + // No need to initialize the tracker here + return false; + } + + // Initialize the tracker the first time we need to use it + delayedDeliveryTracker = Optional.of( + topic.getBrokerService().getDelayedDeliveryTrackerFactory().newTracker(this)); + } + + delayedDeliveryTracker.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis()); + + long deliverAtTime = msgMetadata.hasDeliverAtTime() ? msgMetadata.getDeliverAtTime() : -1L; + return delayedDeliveryTracker.get().addMessage(ledgerId, entryId, deliverAtTime); + } + } + + protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + if (delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().hasMessageAvailable()) { + delayedDeliveryTracker.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis()); + NavigableSet messagesAvailableNow = + delayedDeliveryTracker.get().getScheduledMessages(maxMessagesToRead); + messagesAvailableNow.forEach(p -> redeliveryMessages.add(p.getLedgerId(), p.getEntryId())); + } + + if (!redeliveryMessages.isEmpty()) { + return redeliveryMessages.getMessagesToReplayNow(maxMessagesToRead, position -> true); + } else { + return Collections.emptyNavigableSet(); + } + } + + /** + * This is a mode method designed for Key_Shared mode. + * Filter out the entries that will be discarded due to the order guarantee mechanism of Key_Shared mode. + * This method is in order to avoid the scenario below: + * - Get positions from the Replay queue. + * - Read entries from BK. + * - The order guarantee mechanism of Key_Shared mode filtered out all the entries. + * - Delivery non entry to the client, but we did a BK read. + */ + protected NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + return src; + } + + /** + * This is a mode method designed for Key_Shared mode, to avoid unnecessary stuck. + * See detail {@link PersistentStickyKeyDispatcherMultipleConsumersClassic#hasConsumersNeededNormalRead}. + */ + protected boolean hasConsumersNeededNormalRead() { + return true; + } + + protected synchronized boolean shouldPauseDeliveryForDelayTracker() { + return delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().shouldPauseAllDeliveries(); + } + + @Override + public long getNumberOfDelayedMessages() { + return delayedDeliveryTracker.map(DelayedDeliveryTracker::getNumberOfDelayedMessages).orElse(0L); + } + + @Override + public CompletableFuture clearDelayedMessages() { + if (!topic.isDelayedDeliveryEnabled()) { + return CompletableFuture.completedFuture(null); + } + + if (delayedDeliveryTracker.isPresent()) { + return this.delayedDeliveryTracker.get().clear(); + } else { + DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory = + topic.getBrokerService().getDelayedDeliveryTrackerFactory(); + if (delayedDeliveryTrackerFactory instanceof BucketDelayedDeliveryTrackerFactory + bucketDelayedDeliveryTrackerFactory) { + return bucketDelayedDeliveryTrackerFactory.cleanResidualSnapshots(cursor); + } + return CompletableFuture.completedFuture(null); + } + } + + @Override + public void cursorIsReset() { + if (this.lastIndividualDeletedRangeFromCursorRecovery != null) { + this.lastIndividualDeletedRangeFromCursorRecovery = null; + } + } + + private void addMessageToReplay(Entry entry) { + addMessageToReplay(entry.getLedgerId(), entry.getEntryId()); + entry.release(); + } + + protected boolean addMessageToReplay(long ledgerId, long entryId, long stickyKeyHash) { + if (checkIfMessageIsUnacked(ledgerId, entryId)) { + redeliveryMessages.add(ledgerId, entryId, stickyKeyHash); + return true; + } else { + return false; + } + } + + protected boolean addMessageToReplay(long ledgerId, long entryId) { + if (checkIfMessageIsUnacked(ledgerId, entryId)) { + redeliveryMessages.add(ledgerId, entryId); + return true; + } else { + return false; + } + } + + private boolean checkIfMessageIsUnacked(long ledgerId, long entryId) { + Position markDeletePosition = cursor.getMarkDeletedPosition(); + return (markDeletePosition == null || ledgerId > markDeletePosition.getLedgerId() + || (ledgerId == markDeletePosition.getLedgerId() && entryId > markDeletePosition.getEntryId())); + } + + @Override + public boolean checkAndUnblockIfStuck() { + if (cursor.checkAndUpdateReadPositionChanged()) { + return false; + } + // consider dispatch is stuck if : dispatcher has backlog, available-permits and there is no pending read + if (totalAvailablePermits > 0 && !havePendingReplayRead && !havePendingRead + && cursor.getNumberOfEntriesInBacklog(false) > 0) { + log.warn("{}-{} Dispatcher is stuck and unblocking by issuing reads", topic.getName(), name); + readMoreEntries(); + return true; + } + return false; + } + + public PersistentTopic getTopic() { + return topic; + } + + + public long getDelayedTrackerMemoryUsage() { + return delayedDeliveryTracker.map(DelayedDeliveryTracker::getBufferMemoryUsage).orElse(0L); + } + + public Map getBucketDelayedIndexStats() { + if (delayedDeliveryTracker.isEmpty()) { + return Collections.emptyMap(); + } + + if (delayedDeliveryTracker.get() instanceof BucketDelayedDeliveryTracker) { + return ((BucketDelayedDeliveryTracker) delayedDeliveryTracker.get()).genTopicMetricMap(); + } + + return Collections.emptyMap(); + } + + @Override + public boolean isClassic() { + return true; + } + + public ManagedCursor getCursor() { + return cursor; + } + + protected int getStickyKeyHash(Entry entry) { + return StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; + } + + public Subscription getSubscription() { + return subscription; + } + + public long getNumberOfMessagesInReplay() { + return redeliveryMessages.size(); + } + + @Override + public boolean isHavePendingRead() { + return havePendingRead; + } + + @Override + public boolean isHavePendingReplayRead() { + return havePendingReplayRead; + } + + private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumersClassic.class); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 925e99ed699a2..df053e6d8a549 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -50,6 +50,7 @@ import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; @@ -60,7 +61,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDispatcherMultipleConsumers { +public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDispatcherMultipleConsumers implements + StickyKeyDispatcher { private final boolean allowOutOfOrderDelivery; private final StickyKeyConsumerSelector selector; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java new file mode 100644 index 0000000000000..c227bf5b435bc --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java @@ -0,0 +1,583 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.util.concurrent.FastThreadLocal; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.commons.collections4.MapUtils; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.broker.service.ConsistentHashingStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryAndMetadata; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; +import org.apache.pulsar.broker.service.EntryBatchSizes; +import org.apache.pulsar.broker.service.HashRangeAutoSplitStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.HashRangeExclusiveStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.SendMessageInfo; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.api.proto.KeySharedMode; +import org.apache.pulsar.common.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the "classic" dispatcher implementation for the Key_Shared subscription that was used before + * Pulsar 4.0.0 and PIP-379. When `subscriptionKeySharedUseClassicPersistentImplementation=true`, + * this dispatcher will be used. The main purpose is to provide a way to rollback to the old behavior + * in case of issues with the preferred implementation. + */ +public class PersistentStickyKeyDispatcherMultipleConsumersClassic + extends PersistentDispatcherMultipleConsumersClassic implements StickyKeyDispatcher { + + private final boolean allowOutOfOrderDelivery; + private final StickyKeyConsumerSelector selector; + + private boolean isDispatcherStuckOnReplays = false; + private final KeySharedMode keySharedMode; + + /** + * When a consumer joins, it will be added to this map with the current read position. + * This means that, in order to preserve ordering, new consumers can only receive old + * messages, until the mark-delete position will move past this point. + */ + private final LinkedHashMap recentlyJoinedConsumers; + + PersistentStickyKeyDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, + Subscription subscription, ServiceConfiguration conf, + KeySharedMeta ksm) { + super(topic, cursor, subscription, ksm.isAllowOutOfOrderDelivery()); + + this.allowOutOfOrderDelivery = ksm.isAllowOutOfOrderDelivery(); + this.recentlyJoinedConsumers = allowOutOfOrderDelivery ? null : new LinkedHashMap<>(); + this.keySharedMode = ksm.getKeySharedMode(); + switch (this.keySharedMode) { + case AUTO_SPLIT: + if (conf.isSubscriptionKeySharedUseConsistentHashing()) { + selector = new ConsistentHashingStickyKeyConsumerSelector( + conf.getSubscriptionKeySharedConsistentHashingReplicaPoints(), + false, + // Classic implementation uses Integer.MAX_VALUE - 1 as the range end value + Integer.MAX_VALUE - 1); + } else { + selector = new HashRangeAutoSplitStickyKeyConsumerSelector(); + } + break; + + case STICKY: + this.selector = new HashRangeExclusiveStickyKeyConsumerSelector(); + break; + + default: + throw new IllegalArgumentException("Invalid key-shared mode: " + keySharedMode); + } + } + + @VisibleForTesting + public StickyKeyConsumerSelector getSelector() { + return selector; + } + + @Override + public synchronized CompletableFuture addConsumer(Consumer consumer) { + if (IS_CLOSED_UPDATER.get(this) == TRUE) { + log.warn("[{}] Dispatcher is already closed. Closing consumer {}", name, consumer); + consumer.disconnect(); + return CompletableFuture.completedFuture(null); + } + return super.addConsumer(consumer).thenCompose(__ -> + selector.addConsumer(consumer).handle((result, ex) -> { + if (ex != null) { + synchronized (PersistentStickyKeyDispatcherMultipleConsumersClassic.this) { + consumerSet.removeAll(consumer); + consumerList.remove(consumer); + } + throw FutureUtil.wrapToCompletionException(ex); + } + return result; + }) + ).thenRun(() -> { + synchronized (PersistentStickyKeyDispatcherMultipleConsumersClassic.this) { + Position readPositionWhenJoining = (Position) cursor.getReadPosition(); + consumer.setReadPositionWhenJoining(readPositionWhenJoining); + // If this was the 1st consumer, or if all the messages are already acked, then we + // don't need to do anything special + if (!allowOutOfOrderDelivery + && recentlyJoinedConsumers != null + && consumerList.size() > 1 + && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) { + recentlyJoinedConsumers.put(consumer, readPositionWhenJoining); + } + } + }); + } + + @Override + public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { + // The consumer must be removed from the selector before calling the superclass removeConsumer method. + // In the superclass removeConsumer method, the pending acks that the consumer has are added to + // redeliveryMessages. If the consumer has not been removed from the selector at this point, + // the broker will try to redeliver the messages to the consumer that has already been closed. + // As a result, the messages are not redelivered to any consumer, and the mark-delete position does not move, + // eventually causing all consumers to get stuck. + selector.removeConsumer(consumer); + super.removeConsumer(consumer); + if (recentlyJoinedConsumers != null) { + recentlyJoinedConsumers.remove(consumer); + if (consumerList.size() == 1) { + recentlyJoinedConsumers.clear(); + } + if (removeConsumersFromRecentJoinedConsumers() || !redeliveryMessages.isEmpty()) { + readMoreEntries(); + } + } + } + + private static final FastThreadLocal>> localGroupedEntries = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + + private static final FastThreadLocal>> localGroupedPositions = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + + @Override + protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { + long totalMessagesSent = 0; + long totalBytesSent = 0; + long totalEntries = 0; + int entriesCount = entries.size(); + + // Trigger read more messages + if (entriesCount == 0) { + return true; + } + + if (consumerSet.isEmpty()) { + entries.forEach(Entry::release); + cursor.rewind(); + return false; + } + + // A corner case that we have to retry a readMoreEntries in order to preserver order delivery. + // This may happen when consumer closed. See issue #12885 for details. + if (!allowOutOfOrderDelivery) { + NavigableSet messagesToReplayNow = this.getMessagesToReplayNow(1); + if (messagesToReplayNow != null && !messagesToReplayNow.isEmpty()) { + Position replayPosition = messagesToReplayNow.first(); + + // We have received a message potentially from the delayed tracker and, since we're not using it + // right now, it needs to be added to the redelivery tracker or we won't attempt anymore to + // resend it (until we disconnect consumer). + redeliveryMessages.add(replayPosition.getLedgerId(), replayPosition.getEntryId()); + + if (this.minReplayedPosition != null) { + // If relayPosition is a new entry wither smaller position is inserted for redelivery during this + // async read, it is possible that this relayPosition should dispatch to consumer first. So in + // order to preserver order delivery, we need to discard this read result, and try to trigger a + // replay read, that containing "relayPosition", by calling readMoreEntries. + if (replayPosition.compareTo(minReplayedPosition) < 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] Position {} (<{}) is inserted for relay during current {} read, " + + "discard this read and retry with readMoreEntries.", + name, replayPosition, minReplayedPosition, readType); + } + if (readType == ReadType.Normal) { + entries.forEach(entry -> { + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + }); + } else if (readType == ReadType.Replay) { + entries.forEach(Entry::release); + } + return true; + } + } + } + } + + final Map> groupedEntries = localGroupedEntries.get(); + groupedEntries.clear(); + final Map> consumerStickyKeyHashesMap = new HashMap<>(); + + for (Entry entry : entries) { + int stickyKeyHash = getStickyKeyHash(entry); + Consumer c = selector.select(stickyKeyHash); + if (c != null) { + groupedEntries.computeIfAbsent(c, k -> new ArrayList<>()).add(entry); + consumerStickyKeyHashesMap.computeIfAbsent(c, k -> new HashSet<>()).add(stickyKeyHash); + } else { + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + } + } + + AtomicInteger keyNumbers = new AtomicInteger(groupedEntries.size()); + + int currentThreadKeyNumber = groupedEntries.size(); + if (currentThreadKeyNumber == 0) { + currentThreadKeyNumber = -1; + } + for (Map.Entry> current : groupedEntries.entrySet()) { + Consumer consumer = current.getKey(); + assert consumer != null; // checked when added to groupedEntries + List entriesWithSameKey = current.getValue(); + int entriesWithSameKeyCount = entriesWithSameKey.size(); + int availablePermits = getAvailablePermits(consumer); + int messagesForC = getRestrictedMaxEntriesForConsumer(consumer, + entriesWithSameKey.stream().map(Entry::getPosition).collect(Collectors.toList()), availablePermits, + readType, consumerStickyKeyHashesMap.get(consumer)); + if (log.isDebugEnabled()) { + log.debug("[{}] select consumer {} with messages num {}, read type is {}", + name, consumer.consumerName(), messagesForC, readType); + } + + if (messagesForC < entriesWithSameKeyCount) { + // We are not able to push all the messages with given key to its consumer, + // so we discard for now and mark them for later redelivery + for (int i = messagesForC; i < entriesWithSameKeyCount; i++) { + Entry entry = entriesWithSameKey.get(i); + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + entriesWithSameKey.set(i, null); + } + } + + if (messagesForC > 0) { + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { + for (int i = 0; i < messagesForC; i++) { + Entry entry = entriesWithSameKey.get(i); + redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); + } + } + + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + EntryBatchSizes batchSizes = EntryBatchSizes.get(messagesForC); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); + totalEntries += filterEntriesForConsumer(entriesWithSameKey, batchSizes, sendMessageInfo, + batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + consumer.sendMessages(entriesWithSameKey, batchSizes, batchIndexesAcks, + sendMessageInfo.getTotalMessages(), + sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), + getRedeliveryTracker()).addListener(future -> { + if (future.isDone() && keyNumbers.decrementAndGet() == 0) { + readMoreEntries(); + } + }); + + TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, + -(sendMessageInfo.getTotalMessages() - batchIndexesAcks.getTotalAckedIndexCount())); + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); + } else { + currentThreadKeyNumber = keyNumbers.decrementAndGet(); + } + } + + // acquire message-dispatch permits for already delivered messages + acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); + + if (totalMessagesSent == 0 && (recentlyJoinedConsumers == null || recentlyJoinedConsumers.isEmpty())) { + // This means, that all the messages we've just read cannot be dispatched right now. + // This condition can only happen when: + // 1. We have consumers ready to accept messages (otherwise the would not haven been triggered) + // 2. All keys in the current set of messages are routing to consumers that are currently busy + // + // The solution here is to move on and read next batch of messages which might hopefully contain + // also keys meant for other consumers. + // + // We do it unless that are "recently joined consumers". In that case, we would be looking + // ahead in the stream while the new consumers are not ready to accept the new messages, + // therefore would be most likely only increase the distance between read-position and mark-delete + // position. + isDispatcherStuckOnReplays = true; + return true; + } else if (currentThreadKeyNumber == 0) { + return true; + } + return false; + } + + private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List entries, + int availablePermits, ReadType readType, Set stickyKeyHashes) { + int maxMessages = Math.min(entries.size(), availablePermits); + if (maxMessages == 0) { + return 0; + } + if (readType == ReadType.Normal && stickyKeyHashes != null + && redeliveryMessages.containsStickyKeyHashes(stickyKeyHashes)) { + // If redeliveryMessages contains messages that correspond to the same hash as the messages + // that the dispatcher is trying to send, do not send those messages for order guarantee + return 0; + } + if (recentlyJoinedConsumers == null) { + return maxMessages; + } + removeConsumersFromRecentJoinedConsumers(); + Position maxReadPosition = recentlyJoinedConsumers.get(consumer); + // At this point, all the old messages were already consumed and this consumer + // is now ready to receive any message + if (maxReadPosition == null) { + // The consumer has not recently joined, so we can send all messages + return maxMessages; + } + + // If the read type is Replay, we should avoid send messages that hold by other consumer to the new consumers, + // For example, we have 10 messages [0,1,2,3,4,5,6,7,8,9] + // If the consumer0 get message 0 and 1, and does not acked message 0, then consumer1 joined, + // when consumer1 get message 2,3, the broker will not dispatch messages to consumer1 + // because of the mark delete position did not move forward. + // So message 2,3 will stored in the redeliver tracker. + // Now, consumer2 joined, it will read new messages from the cursor, + // so the recentJoinedPosition is 4 for consumer2 + // Because of there are messages need to redeliver, so the broker will read the redelivery message first [2,3] + // message [2,3] is lower than the recentJoinedPosition 4, + // so the message [2,3] will dispatched to the consumer2 + // But the message [2,3] should not dispatch to consumer2. + + if (readType == ReadType.Replay) { + Position minReadPositionForRecentJoinedConsumer = recentlyJoinedConsumers.values().iterator().next(); + if (minReadPositionForRecentJoinedConsumer != null + && minReadPositionForRecentJoinedConsumer.compareTo(maxReadPosition) < 0) { + maxReadPosition = minReadPositionForRecentJoinedConsumer; + } + } + // Here, the consumer is one that has recently joined, so we can only send messages that were + // published before it has joined. + for (int i = 0; i < maxMessages; i++) { + if (((Position) entries.get(i)).compareTo(maxReadPosition) >= 0) { + // We have already crossed the divider line. All messages in the list are now + // newer than what we can currently dispatch to this consumer + return i; + } + } + + return maxMessages; + } + + @Override + public void markDeletePositionMoveForward() { + // Execute the notification in different thread to avoid a mutex chain here + // from the delete operation that was completed + topic.getBrokerService().getTopicOrderedExecutor().execute(() -> { + synchronized (PersistentStickyKeyDispatcherMultipleConsumersClassic.this) { + if (recentlyJoinedConsumers != null && !recentlyJoinedConsumers.isEmpty() + && removeConsumersFromRecentJoinedConsumers()) { + // After we process acks, we need to check whether the mark-delete position was advanced and we + // can finally read more messages. It's safe to call readMoreEntries() multiple times. + readMoreEntries(); + } + } + }); + } + + private boolean removeConsumersFromRecentJoinedConsumers() { + if (MapUtils.isEmpty(recentlyJoinedConsumers)) { + return false; + } + Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); + boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; + Position mdp = (Position) cursor.getMarkDeletedPosition(); + if (mdp != null) { + Position nextPositionOfTheMarkDeletePosition = + ((ManagedLedgerImpl) cursor.getManagedLedger()).getNextValidPosition(mdp); + while (itr.hasNext()) { + Map.Entry entry = itr.next(); + if (entry.getValue().compareTo(nextPositionOfTheMarkDeletePosition) <= 0) { + itr.remove(); + hasConsumerRemovedFromTheRecentJoinedConsumers = true; + } else { + break; + } + } + } + return hasConsumerRemovedFromTheRecentJoinedConsumers; + } + + @Override + protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + if (isDispatcherStuckOnReplays) { + // If we're stuck on replay, we want to move forward reading on the topic (until the overall max-unacked + // messages kicks in), instead of keep replaying the same old messages, since the consumer that these + // messages are routing to might be busy at the moment + this.isDispatcherStuckOnReplays = false; + return Collections.emptyNavigableSet(); + } else { + return super.getMessagesToReplayNow(maxMessagesToRead); + } + } + + private int getAvailablePermits(Consumer c) { + int availablePermits = Math.max(c.getAvailablePermits(), 0); + if (c.getMaxUnackedMessages() > 0) { + // Avoid negative number + int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + availablePermits = Math.min(availablePermits, remainUnAckedMessages); + } + return availablePermits; + } + + @Override + protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So skip this filter out. + if (isAllowOutOfOrderDelivery()) { + return src; + } + if (src.isEmpty()) { + return src; + } + NavigableSet res = new TreeSet<>(); + // Group positions. + final Map> groupedPositions = localGroupedPositions.get(); + groupedPositions.clear(); + for (Position pos : src) { + Long stickyKeyHash = redeliveryMessages.getHash(pos.getLedgerId(), pos.getEntryId()); + if (stickyKeyHash == null) { + res.add(pos); + continue; + } + Consumer c = selector.select(stickyKeyHash.intValue()); + if (c == null) { + // Maybe using HashRangeExclusiveStickyKeyConsumerSelector. + continue; + } + groupedPositions.computeIfAbsent(c, k -> new ArrayList<>()).add(pos); + } + // Filter positions by the Recently Joined Position rule. + for (Map.Entry> item : groupedPositions.entrySet()) { + int availablePermits = getAvailablePermits(item.getKey()); + if (availablePermits == 0) { + continue; + } + int posCountToRead = getRestrictedMaxEntriesForConsumer(item.getKey(), item.getValue(), availablePermits, + ReadType.Replay, null); + if (posCountToRead > 0) { + res.addAll(item.getValue().subList(0, posCountToRead)); + } + } + return res; + } + + /** + * In Key_Shared mode, the consumer will not receive any entries from a normal reading if it is included in + * {@link #recentlyJoinedConsumers}, they can only receive entries from replay reads. + * If all entries in {@link #redeliveryMessages} have been filtered out due to the order guarantee mechanism, + * Broker need a normal read to make the consumers not included in @link #recentlyJoinedConsumers} will not be + * stuck. See https://github.com/apache/pulsar/pull/7105. + */ + @Override + protected boolean hasConsumersNeededNormalRead() { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So the method "filterOutEntriesWillBeDiscarded" will filter out nothing, just return "true" here. + if (isAllowOutOfOrderDelivery()) { + return true; + } + for (Consumer consumer : consumerList) { + if (consumer == null || consumer.isBlocked()) { + continue; + } + if (recentlyJoinedConsumers.containsKey(consumer)) { + continue; + } + if (consumer.getAvailablePermits() > 0) { + return true; + } + } + return false; + } + + @Override + public SubType getType() { + return SubType.Key_Shared; + } + + @Override + protected Set asyncReplayEntries(Set positions) { + return cursor.asyncReplayEntries(positions, this, ReadType.Replay, true); + } + + public KeySharedMode getKeySharedMode() { + return this.keySharedMode; + } + + public boolean isAllowOutOfOrderDelivery() { + return this.allowOutOfOrderDelivery; + } + + public boolean hasSameKeySharedPolicy(KeySharedMeta ksm) { + return (ksm.getKeySharedMode() == this.keySharedMode + && ksm.isAllowOutOfOrderDelivery() == this.allowOutOfOrderDelivery); + } + + public LinkedHashMap getRecentlyJoinedConsumers() { + return recentlyJoinedConsumers; + } + + public Map> getConsumerKeyHashRanges() { + return selector.getConsumerKeyHashRanges(); + } + + @Override + protected int getStickyKeyHash(Entry entry) { + if (entry instanceof EntryAndMetadata entryAndMetadata) { + // use the cached sticky key hash if available, otherwise calculate the sticky key hash and cache it + return entryAndMetadata.getOrUpdateCachedStickyKeyHash(selector::makeStickyKeyHash); + } + return selector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer())); + } + + private static final Logger log = + LoggerFactory.getLogger(PersistentStickyKeyDispatcherMultipleConsumersClassic.class); + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index b8d351bddf839..eaa147b81b126 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -25,6 +25,7 @@ import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -69,6 +70,7 @@ import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryFilterSupport; import org.apache.pulsar.broker.service.GetStatsOptions; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.plugin.EntryFilter; @@ -250,7 +252,12 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { case Shared: if (dispatcher == null || dispatcher.getType() != SubType.Shared) { previousDispatcher = dispatcher; - dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, this); + ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); + if (config.isSubscriptionSharedUseClassicPersistentImplementation()) { + dispatcher = new PersistentDispatcherMultipleConsumersClassic(topic, cursor, this); + } else { + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, this); + } } break; case Failover: @@ -271,11 +278,19 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { case Key_Shared: KeySharedMeta ksm = consumer.getKeySharedMeta(); if (dispatcher == null || dispatcher.getType() != SubType.Key_Shared - || !((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher) + || !((StickyKeyDispatcher) dispatcher) .hasSameKeySharedPolicy(ksm)) { previousDispatcher = dispatcher; - dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this, - topic.getBrokerService().getPulsar().getConfiguration(), ksm); + ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); + if (config.isSubscriptionKeySharedUseClassicPersistentImplementation()) { + dispatcher = + new PersistentStickyKeyDispatcherMultipleConsumersClassic(topic, cursor, + this, + topic.getBrokerService().getPulsar().getConfiguration(), ksm); + } else { + dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this, + topic.getBrokerService().getPulsar().getConfiguration(), ksm); + } } break; default: @@ -1221,7 +1236,7 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge Dispatcher dispatcher = this.dispatcher; if (dispatcher != null) { Map> consumerKeyHashRanges = getType() == SubType.Key_Shared - ? ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher).getConsumerKeyHashRanges() : null; + ? ((StickyKeyDispatcher) dispatcher).getConsumerKeyHashRanges() : null; dispatcher.getConsumers().forEach(consumer -> { ConsumerStatsImpl consumerStats = consumer.getStats(); if (!getStatsOptions.isExcludeConsumers()) { @@ -1260,17 +1275,18 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge } } - if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (dispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subStats.delayedMessageIndexSizeInBytes = - ((PersistentDispatcherMultipleConsumers) dispatcher).getDelayedTrackerMemoryUsage(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getDelayedTrackerMemoryUsage(); subStats.bucketDelayedIndexStats = - ((PersistentDispatcherMultipleConsumers) dispatcher).getBucketDelayedIndexStats(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getBucketDelayedIndexStats(); } if (Subscription.isIndividualAckMode(subType)) { - if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers d = (PersistentDispatcherMultipleConsumers) dispatcher; + if (dispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers d = + (AbstractPersistentDispatcherMultipleConsumers) dispatcher; subStats.unackedMessages = d.getTotalUnackedMessages(); subStats.blockedSubscriptionOnUnackedMsgs = d.isBlockedDispatcherOnUnackedMsgs(); subStats.msgDelayed = d.getNumberOfDelayedMessages(); @@ -1290,12 +1306,18 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge subStats.isReplicated = isReplicated(); subStats.subscriptionProperties = subscriptionProperties; subStats.isDurable = cursor.isDurable(); - if (getType() == SubType.Key_Shared && dispatcher instanceof PersistentStickyKeyDispatcherMultipleConsumers) { - PersistentStickyKeyDispatcherMultipleConsumers keySharedDispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) dispatcher; - + if (getType() == SubType.Key_Shared && dispatcher instanceof StickyKeyDispatcher) { + StickyKeyDispatcher keySharedDispatcher = (StickyKeyDispatcher) dispatcher; subStats.allowOutOfOrderDelivery = keySharedDispatcher.isAllowOutOfOrderDelivery(); subStats.keySharedMode = keySharedDispatcher.getKeySharedMode().toString(); + + LinkedHashMap recentlyJoinedConsumers = keySharedDispatcher + .getRecentlyJoinedConsumers(); + if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) { + recentlyJoinedConsumers.forEach((k, v) -> { + subStats.consumersAfterMarkDeletePosition.put(k.consumerName(), v.toString()); + }); + } } subStats.nonContiguousDeletedMessagesRanges = cursor.getTotalNonContiguousDeletedMessagesRange(); subStats.nonContiguousDeletedMessagesRangesSerializedSize = 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 3cce175660e70..9c86a99de0f14 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 @@ -2449,9 +2449,9 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats } if (Subscription.isIndividualAckMode(subscription.getType())) { - if (subscription.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) subscription.getDispatcher(); + if (subscription.getDispatcher() instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) subscription.getDispatcher(); topicStatsStream.writePair("blockedSubscriptionOnUnackedMsgs", dispatcher.isBlockedDispatcherOnUnackedMsgs()); topicStatsStream.writePair("unackedMessages", @@ -2758,11 +2758,11 @@ public CompletableFuture getInternalStats(boolean // subscription metrics PersistentSubscription sub = subscriptions.get(Codec.decode(c.getName())); if (sub != null) { - if (sub.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) sub.getDispatcher(); - cs.subscriptionHavePendingRead = dispatcher.havePendingRead; - cs.subscriptionHavePendingReplayRead = dispatcher.havePendingReplayRead; + if (sub.getDispatcher() instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) sub.getDispatcher(); + cs.subscriptionHavePendingRead = dispatcher.isHavePendingRead(); + cs.subscriptionHavePendingReplayRead = dispatcher.isHavePendingReplayRead(); } else if (sub.getDispatcher() instanceof PersistentDispatcherSingleActiveConsumer) { PersistentDispatcherSingleActiveConsumer dispatcher = (PersistentDispatcherSingleActiveConsumer) sub.getDispatcher(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index ffcc3bf0881db..7ed4542b2505f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -36,6 +36,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.BiFunction; +import java.util.stream.Stream; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -207,14 +208,28 @@ public static void receiveMessagesN(BiConsumer, Message> mess * Receive messages concurrently from multiple consumers and handles them using the provided message handler. * * @param messageHandler the message handler - * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages - * @param consumers the consumers to receive messages from - * @param the message value type + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param consumers the consumers to receive messages from + * @param the message value type + */ + public static void receiveMessagesInThreads(BiFunction, Message, Boolean> messageHandler, + final Duration quietTimeout, + Consumer... consumers) { + receiveMessagesInThreads(messageHandler, quietTimeout, Arrays.stream(consumers).sequential()); + } + + /** + * Receive messages concurrently from multiple consumers and handles them using the provided message handler. + * + * @param messageHandler the message handler + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param consumers the consumers to receive messages from + * @param the message value type */ public static void receiveMessagesInThreads(BiFunction, Message, Boolean> messageHandler, final Duration quietTimeout, - Consumer... consumers) { - FutureUtil.waitForAll(Arrays.stream(consumers).sequential().map(consumer -> { + Stream> consumers) { + FutureUtil.waitForAll(consumers.map(consumer -> { return CompletableFuture.runAsync(() -> { try { while (!Thread.currentThread().isInterrupted()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java index ea6ffa2d70dba..cdd0be58b34d5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java @@ -38,7 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.mledger.Position; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; @@ -49,7 +49,7 @@ public abstract class AbstractDeliveryTrackerTest { protected final Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-in-memory-delayed-delivery-test"), 500, TimeUnit.MILLISECONDS); - protected PersistentDispatcherMultipleConsumers dispatcher; + protected AbstractPersistentDispatcherMultipleConsumers dispatcher; protected Clock clock; protected AtomicLong clockTime; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java index 9861ab5723732..ff6bf534129c1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java @@ -24,6 +24,7 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -67,10 +68,10 @@ public void cleanup() throws Exception { @Test public void testFallbackToInMemoryTracker() throws Exception { - Pair pair = + Pair pair = mockDelayedDeliveryTrackerFactoryAndDispatcher(); BrokerService brokerService = pair.getLeft(); - PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); // Since Mocked BucketDelayedDeliveryTrackerFactory.newTracker0() throws RecoverDelayedDeliveryTrackerException, // the factory should be fallback to InMemoryDelayedDeliveryTrackerFactory @@ -83,12 +84,13 @@ public void testFallbackToInMemoryTracker() throws Exception { } - private Pair mockDelayedDeliveryTrackerFactoryAndDispatcher() + private Pair mockDelayedDeliveryTrackerFactoryAndDispatcher() throws Exception { BrokerService brokerService = Mockito.spy(pulsar.getBrokerService()); // Mock dispatcher - PersistentDispatcherMultipleConsumers dispatcher = Mockito.mock(PersistentDispatcherMultipleConsumers.class); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + Mockito.mock(AbstractPersistentDispatcherMultipleConsumers.class); Mockito.doReturn("test").when(dispatcher).getName(); // Mock BucketDelayedDeliveryTrackerFactory @Cleanup @@ -113,10 +115,10 @@ private Pair mockDelayedDe @Test public void testFallbackToInMemoryTrackerFactoryFailed() throws Exception { - Pair pair = + Pair pair = mockDelayedDeliveryTrackerFactoryAndDispatcher(); BrokerService brokerService = pair.getLeft(); - PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); // Mock InMemoryDelayedDeliveryTrackerFactory @Cleanup diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java index 6711aed924c20..ff7763927d888 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java @@ -37,7 +37,7 @@ import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -46,7 +46,7 @@ public class InMemoryDeliveryTrackerTest extends AbstractDeliveryTrackerTest { @DataProvider(name = "delayedTracker") public Object[][] provider(Method method) throws Exception { - dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + dispatcher = mock(AbstractPersistentDispatcherMultipleConsumers.class); clock = mock(Clock.class); clockTime = new AtomicLong(); when(clock.millis()).then(x -> clockTime.get()); @@ -212,7 +212,8 @@ public void testClose() throws Exception { Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-in-memory-delayed-delivery-test"), 1, TimeUnit.MILLISECONDS); - PersistentDispatcherMultipleConsumers dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + mock(AbstractPersistentDispatcherMultipleConsumers.class); AtomicLong clockTime = new AtomicLong(); Clock clock = mock(Clock.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java index bf5a282a4ee6d..426bd50c96bbb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java @@ -51,7 +51,7 @@ import org.apache.pulsar.broker.delayed.AbstractDeliveryTrackerTest; import org.apache.pulsar.broker.delayed.MockBucketSnapshotStorage; import org.apache.pulsar.broker.delayed.MockManagedCursor; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.awaitility.Awaitility; import org.roaringbitmap.RoaringBitmap; import org.roaringbitmap.buffer.ImmutableRoaringBitmap; @@ -74,7 +74,7 @@ public void clean() throws Exception { @DataProvider(name = "delayedTracker") public Object[][] provider(Method method) throws Exception { - dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + dispatcher = mock(AbstractPersistentDispatcherMultipleConsumers.class); clock = mock(Clock.class); clockTime = new AtomicLong(); when(clock.millis()).then(x -> clockTime.get()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java index c66eff2c8a180..2fd288239e362 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java @@ -37,7 +37,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.BatcherBuilder; @@ -786,7 +786,7 @@ public void testConcurrentBatchMessageAck(BatcherBuilder builder) throws Excepti } latch.await(); - PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) topic + AbstractPersistentDispatcherMultipleConsumers dispatcher = (AbstractPersistentDispatcherMultipleConsumers) topic .getSubscription(subscriptionName).getDispatcher(); // check strategically to let ack-message receive by broker retryStrategically((test) -> dispatcher.getConsumers().get(0).getUnackedMessages() == 0, 50, 150); 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 ed7f6974dd26f..21a843a3efc22 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 @@ -41,7 +41,7 @@ import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.BrokerTestUtil; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -106,7 +106,7 @@ public void testBatchMessageAck() { } FutureUtil.waitForAll(sendFutureList).get(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); - PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) topic + AbstractPersistentDispatcherMultipleConsumers dispatcher = (AbstractPersistentDispatcherMultipleConsumers) topic .getSubscription(subscriptionName).getDispatcher(); Message receive1 = consumer.receive(); Message receive2 = consumer.receive(); @@ -515,8 +515,8 @@ private BitSetRecyclable createBitSetRecyclable(int batchSize) { private ManagedCursorImpl getCursor(String topic, String sub) { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); return (ManagedCursorImpl) dispatcher.getCursor(); } @@ -528,8 +528,8 @@ private org.apache.pulsar.broker.service.Consumer makeConsumerReceiveMessagesDel CompletableFuture signal) throws Exception { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); org.apache.pulsar.broker.service.Consumer serviceConsumer = null; for (org.apache.pulsar.broker.service.Consumer c : dispatcher.getConsumers()){ if (c.consumerName().equals(consumerName)) { @@ -664,8 +664,8 @@ public void testPermitsIfHalfAckBatchMessage() throws Exception { private org.apache.pulsar.broker.service.Consumer getTheUniqueServiceConsumer(String topic, String sub) { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService(). getTopic(topic, false).join().get(); - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); return dispatcher.getConsumers().iterator().next(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java index 69f3e2e4d3917..2899e9f2d67db 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java @@ -66,6 +66,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -526,7 +527,8 @@ public void testMultipleDispatcherGetNextConsumerWithDifferentPriorityLevel() th PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, false, 1); Consumer consumer2 = createConsumer(topic, 0, 2, false, 2); Consumer consumer3 = createConsumer(topic, 0, 2, false, 3); @@ -571,7 +573,8 @@ public void testMultipleDispatcherGetNextConsumerWithDifferentPriorityLevel() th public void testFewBlockedConsumerSamePriority() throws Exception{ PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, false, 1); Consumer consumer2 = createConsumer(topic, 0, 2, false, 2); Consumer consumer3 = createConsumer(topic, 0, 2, false, 3); @@ -599,7 +602,8 @@ public void testFewBlockedConsumerSamePriority() throws Exception{ public void testFewBlockedConsumerDifferentPriority() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, false, 1); Consumer consumer2 = createConsumer(topic, 0, 2, false, 2); Consumer consumer3 = createConsumer(topic, 0, 2, false, 3); @@ -654,7 +658,8 @@ public void testFewBlockedConsumerDifferentPriority() throws Exception { public void testFewBlockedConsumerDifferentPriority2() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, true, 1); Consumer consumer2 = createConsumer(topic, 0, 2, true, 2); Consumer consumer3 = createConsumer(topic, 0, 2, true, 3); @@ -677,7 +682,7 @@ public void testFewBlockedConsumerDifferentPriority2() throws Exception { } @SuppressWarnings("unchecked") - private Consumer getNextConsumer(PersistentDispatcherMultipleConsumers dispatcher) throws Exception { + private Consumer getNextConsumer(AbstractPersistentDispatcherMultipleConsumers dispatcher) throws Exception { Consumer consumer = dispatcher.getNextConsumer(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index 1e96da737dd51..92b767104f6cf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -99,6 +99,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; @@ -346,7 +347,7 @@ public void testDispatcherMultiConsumerReadFailed() { when(cursor.getName()).thenReturn("cursor"); Subscription subscription = mock(Subscription.class); when(subscription.getName()).thenReturn("sub"); - PersistentDispatcherMultipleConsumers dispatcher = + AbstractPersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, subscription); dispatcher.readEntriesFailed(new ManagedLedgerException.InvalidCursorPositionException("failed"), null); verify(topic, atLeast(1)).getBrokerService(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 20ea33fb3e1ed..4c8e6897df3fc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -102,8 +102,8 @@ public void testBucketDelayedDeliveryWithAllConsumersDisconnecting() throws Exce Dispatcher dispatcher = pulsar.getBrokerService().getTopicReference(topic).get().getSubscription("sub").getDispatcher(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); List bucketKeys = - ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties().keySet().stream() - .filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties().keySet() + .stream().filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); c1.close(); @@ -117,8 +117,8 @@ public void testBucketDelayedDeliveryWithAllConsumersDisconnecting() throws Exce Dispatcher dispatcher2 = pulsar.getBrokerService().getTopicReference(topic).get().getSubscription("sub").getDispatcher(); List bucketKeys2 = - ((PersistentDispatcherMultipleConsumers) dispatcher2).getCursor().getCursorProperties().keySet().stream() - .filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher2).getCursor().getCursorProperties().keySet() + .stream().filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher2.getNumberOfDelayedMessages(), 1000)); Assert.assertEquals(bucketKeys, bucketKeys2); @@ -152,7 +152,7 @@ public void testUnsubscribe() throws Exception { Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); Map cursorProperties = - ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); List bucketIds = cursorProperties.entrySet().stream() .filter(x -> x.getKey().startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX + "delayed.bucket")).map( x -> Long.valueOf(x.getValue())).toList(); @@ -339,7 +339,7 @@ public void testDelete() throws Exception { Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); Map cursorProperties = - ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); List bucketIds = cursorProperties.entrySet().stream() .filter(x -> x.getKey().startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX + "delayed.bucket")).map( x -> Long.valueOf(x.getValue())).toList(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index 3ca966d210886..e47857e8ec60f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -259,12 +259,14 @@ public void testDelayedDeliveryWithMultipleConcurrentReadEntries() .subscribe(); // Simulate race condition with high frequency of calls to dispatcher.readMoreEntries() - PersistentDispatcherMultipleConsumers d = (PersistentDispatcherMultipleConsumers) ((PersistentTopic) pulsar - .getBrokerService().getTopicReference(topic).get()).getSubscription("shared-sub").getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers d = + (AbstractPersistentDispatcherMultipleConsumers) ((PersistentTopic) pulsar + .getBrokerService().getTopicReference(topic).get()).getSubscription("shared-sub") + .getDispatcher(); Thread t = new Thread(() -> { while (true) { synchronized (d) { - d.readMoreEntries(); + d.readMoreEntriesAsync(); } try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java new file mode 100644 index 0000000000000..487d99891fd3a --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import com.carrotsearch.hppc.ObjectSet; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.awaitility.reflect.WhiteboxImpl; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class PersistentDispatcherMultipleConsumersClassicTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + List consumerList = + WhiteboxImpl.getInternalState(dispatcher, "consumerList"); + + org.apache.pulsar.broker.service.Consumer serviceConsumer = consumerList.get(0); + consumerSet.add(serviceConsumer); + consumerList.add(serviceConsumer); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList2() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + consumerSet.clear(); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } + + @Test + public void testSkipReadEntriesFromCloseCursor() throws Exception { + final String topicName = + BrokerTestUtil.newUniqueName("persistent://public/default/testSkipReadEntriesFromCloseCursor"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + for (int i = 0; i < 10; i++) { + producer.send("message-" + i); + } + producer.close(); + + // Get the dispatcher of the topic. + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService() + .getTopic(topicName, false).join().get(); + + ManagedCursor cursor = Mockito.mock(ManagedCursorImpl.class); + Mockito.doReturn(subscription).when(cursor).getName(); + Subscription sub = Mockito.mock(PersistentSubscription.class); + Mockito.doReturn(topic).when(sub).getTopic(); + // Mock the dispatcher. + PersistentDispatcherMultipleConsumersClassic dispatcher = + Mockito.spy(new PersistentDispatcherMultipleConsumersClassic(topic, cursor, sub)); + // Return 10 permits to make the dispatcher can read more entries. + Mockito.doReturn(10).when(dispatcher).getFirstAvailableConsumerPermits(); + + // Make the count + 1 when call the scheduleReadEntriesWithDelay(...). + AtomicInteger callScheduleReadEntriesWithDelayCnt = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callScheduleReadEntriesWithDelayCnt.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).scheduleReadEntriesWithDelay(Mockito.any(), Mockito.any(), Mockito.anyLong()); + + // Make the count + 1 when call the readEntriesFailed(...). + AtomicInteger callReadEntriesFailed = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callReadEntriesFailed.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).readEntriesFailed(Mockito.any(), Mockito.any()); + + Mockito.doReturn(false).when(cursor).isClosed(); + + // Mock the readEntriesOrWait(...) to simulate the cursor is closed. + Mockito.doAnswer(inv -> { + PersistentDispatcherMultipleConsumersClassic dispatcher1 = inv.getArgument(2); + dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), + null); + return null; + }).when(cursor).asyncReadEntriesOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher), + Mockito.any(), Mockito.any()); + + dispatcher.readMoreEntries(); + + // Verify: the readEntriesFailed should be called once and the scheduleReadEntriesWithDelay should not be called. + Assert.assertTrue(callReadEntriesFailed.get() == 1 && callScheduleReadEntriesWithDelayCnt.get() == 0); + + // Verify: the topic can be deleted successfully. + admin.topics().delete(topicName, false); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java index 052c5ceb5cdde..772b1843d2894 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java @@ -154,7 +154,7 @@ public void testSkipReadEntriesFromCloseCursor() throws Exception { // Mock the readEntriesOrWait(...) to simulate the cursor is closed. Mockito.doAnswer(inv -> { - PersistentDispatcherMultipleConsumers dispatcher1 = inv.getArgument(2); + AbstractPersistentDispatcherMultipleConsumers dispatcher1 = inv.getArgument(2); dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), null); return null; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java new file mode 100644 index 0000000000000..1f40fd46aa344 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java @@ -0,0 +1,482 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.common.protocol.Commands.serializeMetadataAndPayload; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anySet; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelPromise; +import io.netty.channel.EventLoopGroup; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.EntryImpl; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; +import org.apache.pulsar.broker.service.EntryBatchSizes; +import org.apache.pulsar.broker.service.RedeliveryTracker; +import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; +import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.api.proto.KeySharedMode; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.policies.data.HierarchyTopicPolicies; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.protocol.Markers; +import org.awaitility.Awaitility; +import org.mockito.ArgumentCaptor; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class PersistentStickyKeyDispatcherMultipleConsumersClassicTest { + + private PulsarService pulsarMock; + private BrokerService brokerMock; + private ManagedCursorImpl cursorMock; + private Consumer consumerMock; + private PersistentTopic topicMock; + private PersistentSubscription subscriptionMock; + private ServiceConfiguration configMock; + private ChannelPromise channelMock; + private OrderedExecutor orderedExecutor; + + private PersistentStickyKeyDispatcherMultipleConsumersClassic persistentDispatcher; + + final String topicName = "persistent://public/default/testTopic"; + final String subscriptionName = "testSubscription"; + + @BeforeMethod + public void setup() throws Exception { + configMock = mock(ServiceConfiguration.class); + doReturn(true).when(configMock).isSubscriptionRedeliveryTrackerEnabled(); + doReturn(100).when(configMock).getDispatcherMaxReadBatchSize(); + doReturn(true).when(configMock).isSubscriptionKeySharedUseConsistentHashing(); + doReturn(1).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); + doReturn(true).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); + doReturn(false).when(configMock).isAllowOverrideEntryFilters(); + + pulsarMock = mock(PulsarService.class); + doReturn(configMock).when(pulsarMock).getConfiguration(); + + EntryFilterProvider mockEntryFilterProvider = mock(EntryFilterProvider.class); + when(mockEntryFilterProvider.getBrokerEntryFilters()).thenReturn(Collections.emptyList()); + + brokerMock = mock(BrokerService.class); + doReturn(pulsarMock).when(brokerMock).pulsar(); + when(brokerMock.getEntryFilterProvider()).thenReturn(mockEntryFilterProvider); + + HierarchyTopicPolicies topicPolicies = new HierarchyTopicPolicies(); + topicPolicies.getMaxConsumersPerSubscription().updateBrokerValue(0); + + orderedExecutor = OrderedExecutor.newBuilder().build(); + doReturn(orderedExecutor).when(brokerMock).getTopicOrderedExecutor(); + + EventLoopGroup eventLoopGroup = mock(EventLoopGroup.class); + doReturn(eventLoopGroup).when(brokerMock).executor(); + doAnswer(invocation -> { + orderedExecutor.execute(((Runnable)invocation.getArguments()[0])); + return null; + }).when(eventLoopGroup).execute(any(Runnable.class)); + + topicMock = mock(PersistentTopic.class); + doReturn(brokerMock).when(topicMock).getBrokerService(); + doReturn(topicName).when(topicMock).getName(); + doReturn(topicPolicies).when(topicMock).getHierarchyTopicPolicies(); + + cursorMock = mock(ManagedCursorImpl.class); + doReturn(null).when(cursorMock).getLastIndividualDeletedRange(); + doReturn(subscriptionName).when(cursorMock).getName(); + + consumerMock = mock(Consumer.class); + channelMock = mock(ChannelPromise.class); + doReturn("consumer1").when(consumerMock).consumerName(); + doReturn(1000).when(consumerMock).getAvailablePermits(); + doReturn(true).when(consumerMock).isWritable(); + doReturn(channelMock).when(consumerMock).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + + subscriptionMock = mock(PersistentSubscription.class); + when(subscriptionMock.getTopic()).thenReturn(topicMock); + persistentDispatcher = new PersistentStickyKeyDispatcherMultipleConsumersClassic( + topicMock, cursorMock, subscriptionMock, configMock, + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)); + } + + @AfterMethod(alwaysRun = true) + public void cleanup() { + if (persistentDispatcher != null && !persistentDispatcher.isClosed()) { + persistentDispatcher.close(); + } + if (orderedExecutor != null) { + orderedExecutor.shutdownNow(); + orderedExecutor = null; + } + } + + @Test(timeOut = 10000) + public void testAddConsumerWhenClosed() throws Exception { + persistentDispatcher.close().get(); + Consumer consumer = mock(Consumer.class); + persistentDispatcher.addConsumer(consumer); + verify(consumer, times(1)).disconnect(); + assertEquals(0, persistentDispatcher.getConsumers().size()); + assertTrue(persistentDispatcher.getSelector().getConsumerKeyHashRanges().isEmpty()); + } + + @Test + public void testSendMarkerMessage() { + try { + persistentDispatcher.addConsumer(consumerMock); + persistentDispatcher.consumerFlow(consumerMock, 1000); + } catch (Exception e) { + fail("Failed to add mock consumer", e); + } + + List entries = new ArrayList<>(); + ByteBuf markerMessage = Markers.newReplicatedSubscriptionsSnapshotRequest("testSnapshotId", "testSourceCluster"); + entries.add(EntryImpl.create(1, 1, markerMessage)); + entries.add(EntryImpl.create(1, 2, createMessage("message1", 1))); + entries.add(EntryImpl.create(1, 3, createMessage("message2", 2))); + entries.add(EntryImpl.create(1, 4, createMessage("message3", 3))); + entries.add(EntryImpl.create(1, 5, createMessage("message4", 4))); + entries.add(EntryImpl.create(1, 6, createMessage("message5", 5))); + + try { + persistentDispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } catch (Exception e) { + fail("Failed to readEntriesComplete.", e); + } + + Awaitility.await().untilAsserted(() -> { + ArgumentCaptor totalMessagesCaptor = ArgumentCaptor.forClass(Integer.class); + verify(consumerMock, times(1)).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + totalMessagesCaptor.capture(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + + List allTotalMessagesCaptor = totalMessagesCaptor.getAllValues(); + Assert.assertEquals(allTotalMessagesCaptor.get(0).intValue(), 5); + }); + } + + @Test(timeOut = 10000) + public void testSendMessage() { + KeySharedMeta keySharedMeta = new KeySharedMeta().setKeySharedMode(KeySharedMode.STICKY); + PersistentStickyKeyDispatcherMultipleConsumersClassic + persistentDispatcher = new PersistentStickyKeyDispatcherMultipleConsumersClassic( + topicMock, cursorMock, subscriptionMock, configMock, keySharedMeta); + try { + keySharedMeta.addHashRange() + .setStart(0) + .setEnd(9); + + Consumer consumerMock = mock(Consumer.class); + doReturn(keySharedMeta).when(consumerMock).getKeySharedMeta(); + persistentDispatcher.addConsumer(consumerMock); + persistentDispatcher.consumerFlow(consumerMock, 1000); + } catch (Exception e) { + fail("Failed to add mock consumer", e); + } + + List entries = new ArrayList<>(); + entries.add(EntryImpl.create(1, 1, createMessage("message1", 1))); + entries.add(EntryImpl.create(1, 2, createMessage("message2", 2))); + + try { + //Should success,see issue #8960 + persistentDispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } catch (Exception e) { + fail("Failed to readEntriesComplete.", e); + } + } + + @Test + public void testSkipRedeliverTemporally() { + final Consumer slowConsumerMock = mock(Consumer.class); + final ChannelPromise slowChannelMock = mock(ChannelPromise.class); + // add entries to redeliver and read target + final List redeliverEntries = new ArrayList<>(); + redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key1"))); + final List readEntries = new ArrayList<>(); + readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); + readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key2"))); + + try { + Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumersClassic.class.getDeclaredField("totalAvailablePermits"); + totalAvailablePermitsField.setAccessible(true); + totalAvailablePermitsField.set(persistentDispatcher, 1000); + + doAnswer(invocationOnMock -> { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(2)) + .readEntriesComplete(readEntries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + return null; + }).when(cursorMock).asyncReadEntriesOrWait( + anyInt(), anyLong(), any(PersistentStickyKeyDispatcherMultipleConsumersClassic.class), + eq(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal), any()); + } catch (Exception e) { + fail("Failed to set to field", e); + } + + // Create 2Consumers + try { + doReturn("consumer2").when(slowConsumerMock).consumerName(); + // Change slowConsumer availablePermits to 0 and back to normal + when(slowConsumerMock.getAvailablePermits()) + .thenReturn(0) + .thenReturn(1); + doReturn(true).when(slowConsumerMock).isWritable(); + doReturn(slowChannelMock).when(slowConsumerMock).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + + persistentDispatcher.addConsumer(consumerMock); + persistentDispatcher.addConsumer(slowConsumerMock); + } catch (Exception e) { + fail("Failed to add mock consumer", e); + } + + // run PersistentStickyKeyDispatcherMultipleConsumers#sendMessagesToConsumers + // run readMoreEntries internally (and skip internally) + // Change slowConsumer availablePermits to 1 + // run PersistentStickyKeyDispatcherMultipleConsumers#sendMessagesToConsumers internally + // and then stop to dispatch to slowConsumer + if (persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal, + redeliverEntries, true)) { + persistentDispatcher.readMoreEntriesAsync(); + } + + Awaitility.await().untilAsserted(() -> { + verify(consumerMock, times(1)).sendMessages( + argThat(arg -> { + assertEquals(arg.size(), 1); + Entry entry = arg.get(0); + assertEquals(entry.getLedgerId(), 1); + assertEquals(entry.getEntryId(), 3); + return true; + }), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + }); + verify(slowConsumerMock, times(0)).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + } + + @Test(timeOut = 30000) + public void testMessageRedelivery() throws Exception { + final Queue actualEntriesToConsumer1 = new ConcurrentLinkedQueue<>(); + final Queue actualEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); + + final Queue expectedEntriesToConsumer1 = new ConcurrentLinkedQueue<>(); + expectedEntriesToConsumer1.add(PositionFactory.create(1, 1)); + final Queue expectedEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); + expectedEntriesToConsumer2.add(PositionFactory.create(1, 2)); + expectedEntriesToConsumer2.add(PositionFactory.create(1, 3)); + + final AtomicInteger remainingEntriesNum = new AtomicInteger( + expectedEntriesToConsumer1.size() + expectedEntriesToConsumer2.size()); + + // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2 + final List allEntries = new ArrayList<>(); + allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key2"))); + allEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); + allEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key1"))); + allEntries.forEach(entry -> ((EntryImpl) entry).retain()); + + final List redeliverEntries = new ArrayList<>(); + redeliverEntries.add(allEntries.get(0)); // message1 + final List readEntries = new ArrayList<>(); + readEntries.add(allEntries.get(2)); // message3 + + final Consumer consumer1 = mock(Consumer.class); + doReturn("consumer1").when(consumer1).consumerName(); + // Change availablePermits of consumer1 to 0 and then back to normal + when(consumer1.getAvailablePermits()).thenReturn(0).thenReturn(10); + doReturn(true).when(consumer1).isWritable(); + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + List entries = (List) invocationOnMock.getArgument(0); + for (Entry entry : entries) { + remainingEntriesNum.decrementAndGet(); + actualEntriesToConsumer1.add(entry.getPosition()); + } + return channelMock; + }).when(consumer1).sendMessages(anyList(), any(EntryBatchSizes.class), any(EntryBatchIndexesAcks.class), + anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + + final Consumer consumer2 = mock(Consumer.class); + doReturn("consumer2").when(consumer2).consumerName(); + when(consumer2.getAvailablePermits()).thenReturn(10); + doReturn(true).when(consumer2).isWritable(); + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + List entries = (List) invocationOnMock.getArgument(0); + for (Entry entry : entries) { + remainingEntriesNum.decrementAndGet(); + actualEntriesToConsumer2.add(entry.getPosition()); + } + return channelMock; + }).when(consumer2).sendMessages(anyList(), any(EntryBatchSizes.class), any(EntryBatchIndexesAcks.class), + anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + + persistentDispatcher.addConsumer(consumer1); + persistentDispatcher.addConsumer(consumer2); + + final Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumersClassic.class + .getDeclaredField("totalAvailablePermits"); + totalAvailablePermitsField.setAccessible(true); + totalAvailablePermitsField.set(persistentDispatcher, 1000); + + final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumersClassic.class + .getDeclaredField("redeliveryMessages"); + redeliveryMessagesField.setAccessible(true); + MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField + .get(persistentDispatcher); + redeliveryMessages.add(allEntries.get(0).getLedgerId(), allEntries.get(0).getEntryId(), + persistentDispatcher.getStickyKeyHash(allEntries.get(0))); // message1 + redeliveryMessages.add(allEntries.get(1).getLedgerId(), allEntries.get(1).getEntryId(), + persistentDispatcher.getStickyKeyHash(allEntries.get(1))); // message2 + + // Mock Cursor#asyncReplayEntries + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + Set positions = (Set) invocationOnMock.getArgument(0); + List entries = allEntries.stream().filter(entry -> positions.contains(entry.getPosition())) + .collect(Collectors.toList()); + if (!entries.isEmpty()) { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(1)) + .readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Replay); + } + return Collections.emptySet(); + }).when(cursorMock).asyncReplayEntries(anySet(), any(PersistentStickyKeyDispatcherMultipleConsumersClassic.class), + eq(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Replay), anyBoolean()); + + // Mock Cursor#asyncReadEntriesOrWait + AtomicBoolean asyncReadEntriesOrWaitCalled = new AtomicBoolean(); + doAnswer(invocationOnMock -> { + if (asyncReadEntriesOrWaitCalled.compareAndSet(false, true)) { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(2)) + .readEntriesComplete(readEntries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } else { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(2)) + .readEntriesComplete(Collections.emptyList(), PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } + return null; + }).when(cursorMock).asyncReadEntriesOrWait(anyInt(), anyLong(), + any(PersistentStickyKeyDispatcherMultipleConsumersClassic.class), + eq(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal), any()); + + // (1) Run sendMessagesToConsumers + // (2) Attempts to send message1 to consumer1 but skipped because availablePermits is 0 + // (3) Change availablePermits of consumer1 to 10 + // (4) Run readMoreEntries internally + // (5) Run sendMessagesToConsumers internally + // (6) Attempts to send message3 to consumer2 but skipped because redeliveryMessages contains message2 + persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Replay, + redeliverEntries, true); + while (remainingEntriesNum.get() > 0) { + // (7) Run readMoreEntries and resend message1 to consumer1 and message2-3 to consumer2 + persistentDispatcher.readMoreEntries(); + } + + assertThat(actualEntriesToConsumer1).containsExactlyElementsOf(expectedEntriesToConsumer1); + assertThat(actualEntriesToConsumer2).containsExactlyElementsOf(expectedEntriesToConsumer2); + + allEntries.forEach(entry -> entry.release()); + } + + private ByteBuf createMessage(String message, int sequenceId) { + return createMessage(message, sequenceId, "testKey"); + } + + private ByteBuf createMessage(String message, int sequenceId, String key) { + MessageMetadata messageMetadata = new MessageMetadata() + .setSequenceId(sequenceId) + .setProducerName("testProducer") + .setPartitionKey(key) + .setPartitionKeyB64Encoded(false) + .setPublishTime(System.currentTimeMillis()); + return serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, Unpooled.copiedBuffer(message.getBytes(UTF_8))); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index 4b29ead984e7a..7234f0caefc63 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -720,7 +720,7 @@ public void testNoBackoffDelayWhenDelayedMessages(boolean dispatchMessagesInSubs AtomicInteger reScheduleReadInMsCalled = new AtomicInteger(0); AtomicBoolean delayAllMessages = new AtomicBoolean(true); - PersistentDispatcherMultipleConsumers dispatcher; + AbstractPersistentDispatcherMultipleConsumers dispatcher; if (isKeyShared) { dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers( topicMock, cursorMock, subscriptionMock, configMock, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index 14403765105b9..5b2998216e8e1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -233,7 +233,7 @@ public void testConsumerStatsOutput() throws Exception { "unackedMessages", "avgMessagesPerEntry", "blockedConsumerOnUnackedMsgs", - "lastSentPositionWhenJoining", + "readPositionWhenJoining", "lastAckedTime", "lastAckedTimestamp", "lastConsumedTime", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 2b16647f5590c..08efb6d9583ef 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -65,10 +65,13 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.DrainingHashesTracker; import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; @@ -79,6 +82,7 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.tests.KeySharedImplementationType; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.mockito.Mockito; @@ -89,53 +93,80 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker-impl") public class KeySharedSubscriptionTest extends ProducerConsumerBase { - private static final Logger log = LoggerFactory.getLogger(KeySharedSubscriptionTest.class); private static final List keys = Arrays.asList("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"); private static final String SUBSCRIPTION_NAME = "key_shared"; + private final KeySharedImplementationType implementationType; + + // Comment out the next line (Factory annotation) to run tests manually in IntelliJ, one-by-one + @Factory + public static Object[] createTestInstances() { + return KeySharedImplementationType.generateTestInstances(KeySharedSubscriptionTest::new); + } + + public KeySharedSubscriptionTest() { + // set the default implementation type for manual running in IntelliJ + this(KeySharedImplementationType.DEFAULT); + } + + public KeySharedSubscriptionTest(KeySharedImplementationType implementationType) { + this.implementationType = implementationType; + } + + private Object[][] prependImplementationTypeToData(Object[][] data) { + return implementationType.prependImplementationTypeToData(data); + } + + @DataProvider(name = "currentImplementationType") + public Object[] currentImplementationType() { + return new Object[]{ implementationType }; + } @DataProvider(name = "batch") - public Object[] batchProvider() { - return new Object[] { - false, - true - }; + public Object[][] batchProvider() { + return prependImplementationTypeToData(new Object[][]{ + {false}, + {true} + }); } @DataProvider(name = "partitioned") public Object[][] partitionedProvider() { - return new Object[][] { - { false }, - { true } - }; + return prependImplementationTypeToData(new Object[][]{ + {false}, + {true} + }); } @DataProvider(name = "data") public Object[][] dataProvider() { - return new Object[][] { + return prependImplementationTypeToData(new Object[][]{ // Topic-Type and "Batching" - { "persistent", false }, - { "persistent", true }, - { "non-persistent", false }, - { "non-persistent", true }, - }; + {"persistent", false}, + {"persistent", true}, + {"non-persistent", false}, + {"non-persistent", true}, + }); } @DataProvider(name = "topicDomain") public Object[][] topicDomainProvider() { - return new Object[][] { - { "persistent" }, - { "non-persistent" } - }; + return prependImplementationTypeToData(new Object[][]{ + {"persistent"}, + {"non-persistent"} + }); } @BeforeClass(alwaysRun = true) @Override protected void setup() throws Exception { + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(implementationType.classic); + conf.setSubscriptionSharedUseClassicPersistentImplementation(implementationType.classic); this.conf.setUnblockStuckSubscriptionEnabled(true); super.internalSetup(); super.producerBaseSetup(); @@ -170,7 +201,8 @@ public void resetDefaultNamespace() throws Exception { private static final int NUMBER_OF_KEYS = 300; @Test(dataProvider = "data") - public void testSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector(String topicType, boolean enableBatch) + public void testSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector(KeySharedImplementationType impl, + String topicType, boolean enableBatch) throws PulsarClientException { String topic = topicType + "://public/default/key_shared-" + UUID.randomUUID(); @@ -197,7 +229,7 @@ public void testSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector(St } @Test(dataProvider = "data") - public void testSendAndReceiveWithBatching(String topicType, boolean enableBatch) throws Exception { + public void testSendAndReceiveWithBatching(KeySharedImplementationType impl, String topicType, boolean enableBatch) throws Exception { String topic = topicType + "://public/default/key_shared-" + UUID.randomUUID(); @Cleanup @@ -242,7 +274,9 @@ public void testSendAndReceiveWithBatching(String topicType, boolean enableBatch } @Test(dataProvider = "batch") - public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) throws PulsarClientException { + public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) + throws PulsarClientException { String topic = "persistent://public/default/key_shared_exclusive-" + UUID.randomUUID(); @Cleanup @@ -294,8 +328,9 @@ public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(bo @Test(dataProvider = "data") public void testConsumerCrashSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( - String topicType, - boolean enableBatch + KeySharedImplementationType impl, + String topicType, + boolean enableBatch ) throws PulsarClientException, InterruptedException { String topic = topicType + "://public/default/key_shared_consumer_crash-" + UUID.randomUUID(); @@ -338,8 +373,9 @@ public void testConsumerCrashSendAndReceiveWithHashRangeAutoSplitStickyKeyConsum @Test(dataProvider = "data") public void testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( - String topicType, - boolean enableBatch + KeySharedImplementationType impl, + String topicType, + boolean enableBatch ) throws PulsarClientException { String topic = topicType + "://public/default/key_shared_no_key-" + UUID.randomUUID(); @@ -365,7 +401,8 @@ public void testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelect } @Test(dataProvider = "batch") - public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) + public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) throws PulsarClientException { String topic = "persistent://public/default/key_shared_no_key_exclusive-" + UUID.randomUUID(); @@ -415,7 +452,8 @@ public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelect } @Test(dataProvider = "batch") - public void testOrderingKeyWithHashRangeAutoSplitStickyKeyConsumerSelector(boolean enableBatch) + public void testOrderingKeyWithHashRangeAutoSplitStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) throws PulsarClientException { String topic = "persistent://public/default/key_shared_ordering_key-" + UUID.randomUUID(); @@ -443,7 +481,8 @@ public void testOrderingKeyWithHashRangeAutoSplitStickyKeyConsumerSelector(boole } @Test(dataProvider = "batch") - public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) + public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) throws PulsarClientException { String topic = "persistent://public/default/key_shared_exclusive_ordering_key-" + UUID.randomUUID(); @@ -512,8 +551,8 @@ public void testDisableKeySharedSubscription() throws PulsarClientException { } } - @Test - public void testCannotUseAcknowledgeCumulative() throws PulsarClientException { + @Test(dataProvider = "currentImplementationType") + public void testCannotUseAcknowledgeCumulative(KeySharedImplementationType impl) throws PulsarClientException { String topic = "persistent://public/default/key_shared_ack_cumulative-" + UUID.randomUUID(); @Cleanup @@ -538,7 +577,7 @@ public void testCannotUseAcknowledgeCumulative() throws PulsarClientException { } @Test(dataProvider = "batch") - public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exception { + public void testMakingProgressWithSlowerConsumer(KeySharedImplementationType impl, boolean enableBatch) throws Exception { String topic = "testMakingProgressWithSlowerConsumer-" + UUID.randomUUID(); String slowKey = "slowKey"; @@ -620,8 +659,8 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc } } - @Test - public void testOrderingWhenAddingConsumers() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testOrderingWhenAddingConsumers(KeySharedImplementationType impl) throws Exception { String topic = "testOrderingWhenAddingConsumers-" + UUID.randomUUID(); int numberOfKeys = 10; @@ -668,13 +707,13 @@ public void testOrderingWhenAddingConsumers() throws Exception { } @SneakyThrows - private PersistentStickyKeyDispatcherMultipleConsumers getDispatcher(String topic, String subscription) { - return (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get() + private StickyKeyDispatcher getDispatcher(String topic, String subscription) { + return (StickyKeyDispatcher) pulsar.getBrokerService().getTopicIfExists(topic).get() .get().getSubscription(subscription).getDispatcher(); } - @Test - public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testReadAheadWithConfiguredLookAheadLimit(KeySharedImplementationType impl) throws Exception { String topic = "testReadAheadWithConfiguredLookAheadLimit-" + UUID.randomUUID(); // Set the look ahead limit to 50 for subscriptions @@ -730,8 +769,8 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { assertTrue(entryId < 100); } - @Test - public void testRemoveFirstConsumer() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testRemoveFirstConsumer(KeySharedImplementationType impl) throws Exception { String topic = "testReadAheadWhenAddingConsumers-" + UUID.randomUUID(); int numberOfKeys = 10; @@ -788,8 +827,8 @@ public void testRemoveFirstConsumer() throws Exception { } } - @Test - public void testHashRangeConflict() throws PulsarClientException { + @Test(dataProvider = "currentImplementationType") + public void testHashRangeConflict(KeySharedImplementationType impl) throws PulsarClientException { final String topic = "persistent://public/default/testHashRangeConflict-" + UUID.randomUUID().toString(); final String sub = "test"; @@ -799,7 +838,7 @@ public void testHashRangeConflict() throws PulsarClientException { Consumer consumer2 = createFixedHashRangesConsumer(topic, sub, Range.of(100,399)); Assert.assertTrue(consumer2.isConnected()); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, sub); + StickyKeyDispatcher dispatcher = getDispatcher(topic, sub); Assert.assertEquals(dispatcher.getConsumers().size(), 2); try { @@ -849,8 +888,8 @@ public void testHashRangeConflict() throws PulsarClientException { Assert.assertFalse(dispatcher.isConsumerConnected()); } - @Test - public void testWithMessageCompression() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testWithMessageCompression(KeySharedImplementationType impl) throws Exception { final String topic = "testWithMessageCompression" + UUID.randomUUID().toString(); Producer producer = pulsarClient.newProducer() .topic(topic) @@ -876,8 +915,8 @@ public void testWithMessageCompression() throws Exception { consumer.close(); } - @Test - public void testAttachKeyToMessageMetadata() throws PulsarClientException { + @Test(dataProvider = "currentImplementationType") + public void testAttachKeyToMessageMetadata(KeySharedImplementationType impl) throws PulsarClientException { String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); @Cleanup @@ -904,8 +943,8 @@ public void testAttachKeyToMessageMetadata() throws PulsarClientException { receiveAndCheckDistribution(Lists.newArrayList(consumer1, consumer2, consumer3), 1000); } - @Test - public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testContinueDispatchMessagesWhenMessageTTL(KeySharedImplementationType impl) throws Exception { int defaultTTLSec = 3; int totalMessages = 1000; int numberOfKeys = 50; @@ -922,7 +961,7 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subName); + StickyKeyDispatcher dispatcher = getDispatcher(topic, subName); StickyKeyConsumerSelector selector = dispatcher.getSelector(); @Cleanup @@ -964,7 +1003,7 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { if (received != null) { int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); DrainingHashesTracker.DrainingHashEntry entry = - dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + !impl.classic ? getDrainingHashesTracker(dispatcher).getEntry(stickyKeyHash) : null; Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + "included in blockedHashes=%s", received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); @@ -982,10 +1021,10 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { received = consumer3.receive(1, TimeUnit.SECONDS); } catch (PulsarClientException ignore) { } - if (received != null) { + if (received != null && !impl.classic) { int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); DrainingHashesTracker.DrainingHashEntry entry = - dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + !impl.classic ? getDrainingHashesTracker(dispatcher).getEntry(stickyKeyHash) : null; Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + "included in blockedHashes=%s", received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); @@ -1018,8 +1057,12 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { count -> assertThat(count.get()).isGreaterThan(0)); } + private DrainingHashesTracker getDrainingHashesTracker(Dispatcher dispatcher) { + return ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher).getDrainingHashesTracker(); + } + @Test(dataProvider = "partitioned") - public void testOrderingWithConsumerListener(boolean partitioned) throws Exception { + public void testOrderingWithConsumerListener(KeySharedImplementationType impl, boolean partitioned) throws Exception { final String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); if (partitioned) { admin.topics().createPartitionedTopic(topic, 3); @@ -1075,8 +1118,8 @@ public void testOrderingWithConsumerListener(boolean partitioned) throws Excepti consumer.close(); } - @Test - public void testKeySharedConsumerWithEncrypted() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testKeySharedConsumerWithEncrypted(KeySharedImplementationType impl) throws Exception { final String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); final int totalMessages = 100; @@ -1142,7 +1185,7 @@ public void testKeySharedConsumerWithEncrypted() throws Exception { } @Test(dataProvider = "topicDomain") - public void testSelectorChangedAfterAllConsumerDisconnected(String topicDomain) throws PulsarClientException, + public void testSelectorChangedAfterAllConsumerDisconnected(KeySharedImplementationType impl, String topicDomain) throws PulsarClientException, ExecutionException, InterruptedException { final String topicName = TopicName.get(topicDomain, "public", "default", "testSelectorChangedAfterAllConsumerDisconnected" + UUID.randomUUID()).toString(); @@ -1187,8 +1230,8 @@ public void testSelectorChangedAfterAllConsumerDisconnected(String topicDomain) consumer1.close(); } - @Test - public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected(KeySharedImplementationType impl) throws Exception { final String topicName = "persistent://public/default/change-allow-ooo-delivery-" + UUID.randomUUID(); final String subName = "my-sub"; @@ -1207,7 +1250,7 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr producer.send("message".getBytes()); Awaitility.await().untilAsserted(() -> assertNotNull(consumer1.receive(100, TimeUnit.MILLISECONDS))); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topicName, subName); + StickyKeyDispatcher dispatcher = getDispatcher(topicName, subName); assertTrue(dispatcher.isAllowOutOfOrderDelivery()); consumer1.close(); @@ -1225,8 +1268,8 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr consumer2.close(); } - @Test(timeOut = 30_000) - public void testCheckConsumersWithSameName() throws Exception { + @Test(timeOut = 30_000, dataProvider = "currentImplementationType") + public void testCheckConsumersWithSameName(KeySharedImplementationType impl) throws Exception { final String topicName = "persistent://public/default/same-name-" + UUID.randomUUID(); final String subName = "my-sub"; final String consumerName = "name"; @@ -1270,25 +1313,37 @@ public void testCheckConsumersWithSameName() throws Exception { @Cleanup("shutdownNow") ExecutorService e = Executors.newCachedThreadPool(); e.submit(() -> { - while (l.getCount() > 0) { + while (l.getCount() > 0 && !Thread.currentThread().isInterrupted()) { try { Message msg = c2.receive(1, TimeUnit.SECONDS); + if (msg == null) { + continue; + } c2.acknowledge(msg); l.countDown(); } catch (PulsarClientException ex) { ex.printStackTrace(); + if (ex instanceof PulsarClientException.AlreadyClosedException) { + break; + } } } }); e.submit(() -> { - while (l.getCount() > 0) { + while (l.getCount() > 0 && !Thread.currentThread().isInterrupted()) { try { Message msg = c3.receive(1, TimeUnit.SECONDS); + if (msg == null) { + continue; + } c3.acknowledge(msg); l.countDown(); } catch (PulsarClientException ex) { ex.printStackTrace(); + if (ex instanceof PulsarClientException.AlreadyClosedException) { + break; + } } } }); @@ -1303,7 +1358,7 @@ private Object[][] preSendProvider() { private KeySharedMode getKeySharedModeOfSubscription(Topic topic, String subscription) { if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.persistent)) { - return ((PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subscription) + return ((StickyKeyDispatcher) topic.getSubscription(subscription) .getDispatcher()).getKeySharedMode(); } else if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.non_persistent)) { return ((NonPersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subscription) @@ -1390,45 +1445,37 @@ private void receive(List> consumers) throws PulsarClientException { */ private void receiveAndCheckDistribution(List> consumers, int expectedTotalMessage) throws PulsarClientException { // Add a key so that we know this key was already assigned to one consumer - Map> keyToConsumer = new HashMap<>(); - Map, Integer> messagesPerConsumer = new HashMap<>(); + Map> keyToConsumer = new ConcurrentHashMap<>(); + Map, AtomicInteger> messagesPerConsumer = new ConcurrentHashMap<>(); + AtomicInteger totalMessages = new AtomicInteger(); - int totalMessages = 0; + BiFunction, Message, Boolean> messageHandler = (consumer, msg) -> { + totalMessages.incrementAndGet(); + messagesPerConsumer.computeIfAbsent(consumer, k -> new AtomicInteger()).incrementAndGet(); + try { + consumer.acknowledge(msg); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } - for (Consumer c : consumers) { - int messagesForThisConsumer = 0; - while (true) { - Message msg = c.receive(100, TimeUnit.MILLISECONDS); - if (msg == null) { - // Go to next consumer - messagesPerConsumer.put(c, messagesForThisConsumer); - break; - } - - ++totalMessages; - ++messagesForThisConsumer; - c.acknowledge(msg); - - if (msg.hasKey() || msg.hasOrderingKey()) { - String key = msg.hasOrderingKey() ? new String(msg.getOrderingKey()) : msg.getKey(); - Consumer assignedConsumer = keyToConsumer.get(key); - if (assignedConsumer == null) { - // This is a new key - keyToConsumer.put(key, c); - } else { - // The consumer should be the same - assertEquals(c, assignedConsumer); - } + if (msg.hasKey() || msg.hasOrderingKey()) { + String key = msg.hasOrderingKey() ? new String(msg.getOrderingKey()) : msg.getKey(); + Consumer assignedConsumer = keyToConsumer.putIfAbsent(key, consumer); + if (assignedConsumer != null && !assignedConsumer.equals(consumer)) { + assertEquals(consumer, assignedConsumer); } } - } + return true; + }; - final double PERCENT_ERROR = 0.40; // 40 % + BrokerTestUtil.receiveMessagesInThreads(messageHandler, Duration.ofMillis(250), + consumers.stream().map(Consumer.class::cast)); - double expectedMessagesPerConsumer = totalMessages / consumers.size(); - Assert.assertEquals(expectedTotalMessage, totalMessages); - for (int count : messagesPerConsumer.values()) { - Assert.assertEquals(count, expectedMessagesPerConsumer, expectedMessagesPerConsumer * PERCENT_ERROR); + final double PERCENT_ERROR = 0.40; // 40 % + double expectedMessagesPerConsumer = totalMessages.get() / (double) consumers.size(); + Assert.assertEquals(expectedTotalMessage, totalMessages.get()); + for (AtomicInteger count : messagesPerConsumer.values()) { + Assert.assertEquals(count.get(), expectedMessagesPerConsumer, expectedMessagesPerConsumer * PERCENT_ERROR); } } @@ -1531,8 +1578,8 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe } } - @Test - public void testStickyKeyRangesRestartConsumers() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testStickyKeyRangesRestartConsumers(KeySharedImplementationType impl) throws Exception { final String topic = TopicName.get("persistent", "public", "default", "testStickyKeyRangesRestartConsumers" + UUID.randomUUID()).toString(); @@ -1663,8 +1710,8 @@ public void testStickyKeyRangesRestartConsumers() throws Exception { producerFuture.get(); } - @Test - public void testContinueDispatchMessagesWhenMessageDelayed() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testContinueDispatchMessagesWhenMessageDelayed(KeySharedImplementationType impl) throws Exception { int delayedMessages = 40; int messages = 40; int sum = 0; @@ -1765,8 +1812,8 @@ private AtomicInteger injectReplayReadCounter(String topicName, String cursorNam return replyReadCounter; } - @Test - public void testNoRepeatedReadAndDiscard() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testNoRepeatedReadAndDiscard(KeySharedImplementationType impl) throws Exception { int delayedMessages = 100; int numberOfKeys = delayedMessages; final String topic = newUniqueName("persistent://public/default/tp"); @@ -1839,10 +1886,10 @@ public void testNoRepeatedReadAndDiscard() throws Exception { @DataProvider(name = "allowKeySharedOutOfOrder") public Object[][] allowKeySharedOutOfOrder() { - return new Object[][]{ + return prependImplementationTypeToData(new Object[][]{ {true}, {false} - }; + }); } /** @@ -1860,7 +1907,7 @@ public Object[][] allowKeySharedOutOfOrder() { * - at last, all messages will be received. */ @Test(timeOut = 180 * 1000, dataProvider = "allowKeySharedOutOfOrder") // the test will be finished in 60s. - public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedOutOfOrder) throws Exception { + public void testRecentJoinedPosWillNotStuckOtherConsumer(KeySharedImplementationType impl, boolean allowKeySharedOutOfOrder) throws Exception { final int messagesSentPerTime = 100; final Set totalReceivedMessages = new TreeSet<>(); final String topic = newUniqueName("persistent://public/default/tp"); @@ -2020,8 +2067,10 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO admin.topics().delete(topic, false); } - @Test - public void testReadAheadLimit() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testReadAheadLimit(KeySharedImplementationType impl) throws Exception { + // skip for classic implementation since the feature is not implemented + impl.skipIfClassic(); String topic = "testReadAheadLimit-" + UUID.randomUUID(); int numberOfKeys = 1000; long pauseTime = 100L; @@ -2040,7 +2089,7 @@ public void testReadAheadLimit() throws Exception { .subscribe() .close(); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subscriptionName); + StickyKeyDispatcher dispatcher = getDispatcher(topic, subscriptionName); // create a function to use for checking the number of messages in replay Runnable checkLimit = () -> { @@ -2145,16 +2194,18 @@ public void testReadAheadLimit() throws Exception { private StickyKeyConsumerSelector getSelector(String topic, String subscription) { Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + StickyKeyDispatcher dispatcher = (StickyKeyDispatcher) sub.getDispatcher(); return dispatcher.getSelector(); } // This test case simulates a rolling restart scenario with behaviors that can trigger out-of-order issues. // In earlier versions of Pulsar, this issue occurred in about 25% of cases. // To increase the probability of reproducing the issue, use the invocationCount parameter. - @Test//(invocationCount = 50) - public void testOrderingAfterReconnects() throws Exception { + @Test(dataProvider = "currentImplementationType")//(invocationCount = 50) + public void testOrderingAfterReconnects(KeySharedImplementationType impl) throws Exception { + // skip for classic implementation since this fails + impl.skipIfClassic(); + String topic = newUniqueName("testOrderingAfterReconnects"); int numberOfKeys = 1000; long pauseTime = 100L; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index 02de11a2bcc95..ce554ab2d9c00 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -93,7 +93,7 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -180,7 +180,7 @@ public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(Subscriptio DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -264,7 +264,7 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -356,7 +356,7 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -442,7 +442,7 @@ private void testDispatchRate(SubscriptionType subscription, DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -649,7 +649,7 @@ public void testRateLimitingMultipleConsumers() throws Exception { DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -805,7 +805,7 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName1).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -855,7 +855,7 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { .subscribe(); subDispatcher = topic2.getSubscription(subName2).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java index 390e81ad664f9..8fe1f3e58d96d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java @@ -32,6 +32,7 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -104,8 +105,8 @@ private void triggerNewReadMoreEntries(String tpName, String cursorName) throws PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get(); Dispatcher dispatcher = persistentTopic.getSubscription(cursorName).getDispatcher(); - if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { - ((PersistentDispatcherMultipleConsumers) dispatcher).readMoreEntries(); + if (dispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).readMoreEntriesAsync(); } else if (dispatcher instanceof PersistentDispatcherSingleActiveConsumer) { PersistentDispatcherSingleActiveConsumer persistentDispatcherSingleActiveConsumer = ((PersistentDispatcherSingleActiveConsumer) dispatcher); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionMaxUnackedMessagesTest.java similarity index 85% rename from pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java rename to pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionMaxUnackedMessagesTest.java index 704af89777f05..b3ef641ca1979 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionMaxUnackedMessagesTest.java @@ -34,8 +34,8 @@ import java.util.concurrent.atomic.AtomicLong; import lombok.SneakyThrows; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.Consumer; @@ -47,19 +47,39 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.tests.KeySharedImplementationType; import org.awaitility.Awaitility; import org.testng.Assert; 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-impl") -public class KeySharedSubscriptionTest extends ProducerConsumerBase { +public class KeySharedSubscriptionMaxUnackedMessagesTest extends ProducerConsumerBase { + private final KeySharedImplementationType implementationType; + + // Comment out the next line (Factory annotation) to run tests manually in IntelliJ, one-by-one + @Factory + public static Object[] createTestInstances() { + return KeySharedImplementationType.generateTestInstances(KeySharedSubscriptionMaxUnackedMessagesTest::new); + } + + public KeySharedSubscriptionMaxUnackedMessagesTest() { + // set the default implementation type for manual running in IntelliJ + this(KeySharedImplementationType.DEFAULT); + } + + public KeySharedSubscriptionMaxUnackedMessagesTest(KeySharedImplementationType implementationType) { + this.implementationType = implementationType; + } @Override @BeforeMethod protected void setup() throws Exception { + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(implementationType.classic); + conf.setSubscriptionSharedUseClassicPersistentImplementation(implementationType.classic); conf.setMaxUnackedMessagesPerConsumer(10); super.internalSetup(); super.producerBaseSetup(); @@ -82,16 +102,17 @@ enum KeySharedSelectorType { @DataProvider public Object[][] subType() { - return new Object[][] { - { SubscriptionType.Shared, null }, - { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_ConsistentHashing }, - { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_Classic }, - { SubscriptionType.Key_Shared, KeySharedSelectorType.Sticky } - }; + return implementationType.prependImplementationTypeToData(new Object[][]{ + {SubscriptionType.Shared, null}, + {SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_ConsistentHashing}, + {SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_Classic}, + {SubscriptionType.Key_Shared, KeySharedSelectorType.Sticky} + }); } @Test(dataProvider = "subType", timeOut = 30000) - public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType, + public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(KeySharedImplementationType impl, + SubscriptionType subscriptionType, KeySharedSelectorType selectorType) throws PulsarClientException { if (selectorType == KeySharedSelectorType.AutoSplit_Classic) { @@ -258,8 +279,7 @@ private static void waitUntilLastActiveTimeNoLongerGetsUpdated(AtomicLong lastAc private StickyKeyConsumerSelector getSelector(String topic, String subscription) { Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + StickyKeyDispatcher dispatcher = (StickyKeyDispatcher) sub.getDispatcher(); return dispatcher.getSelector(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java b/pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java new file mode 100644 index 0000000000000..39b504131fcee --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java @@ -0,0 +1,61 @@ +/* + * 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.tests; + +import java.util.Arrays; +import java.util.function.Function; +import org.testng.SkipException; + +/** + * KeyShared implementation type used in test. + */ +public enum KeySharedImplementationType { + // default implementation, PIP-379 + PIP379(false), + // classic implementation before PIP-282 and PIP-379 + Classic(true); + + public static final KeySharedImplementationType DEFAULT = PIP379; + public final boolean classic; + + KeySharedImplementationType(boolean classic) { + this.classic = classic; + } + + public void skipIfClassic() { + if (classic) { + throw new SkipException("Test is not applicable for classic implementation"); + } + } + + public Object[][] prependImplementationTypeToData(Object[][] data) { + return Arrays.stream(data) + .map(array -> { + Object[] newArray = new Object[array.length + 1]; + newArray[0] = this; + System.arraycopy(array, 0, newArray, 1, array.length); + return newArray; + }) + .toArray(Object[][]::new); + } + + public static Object[] generateTestInstances(Function testInstanceFactory) { + return Arrays.stream(KeySharedImplementationType.values()).map(testInstanceFactory).toArray(); + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java index 5f2cf7b209ee9..d2d3600df96ed 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java @@ -72,8 +72,8 @@ public interface ConsumerStats { /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ boolean isBlockedConsumerOnUnackedMsgs(); - /** The last sent position of the cursor when the consumer joining. */ - String getLastSentPositionWhenJoining(); + /** The read position of the cursor when the consumer joining. */ + String getReadPositionWhenJoining(); /** Address of this consumer. */ String getAddress(); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index 7b7c1f5765cc5..ce3a080a855da 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -118,6 +118,9 @@ public interface SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ String getKeySharedMode(); + /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ + Map getConsumersAfterMarkDeletePosition(); + /** SubscriptionProperties (key/value strings) associated with this subscribe. */ Map getSubscriptionProperties(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java index b4c5d21e6926e..de36b330b7f1a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java @@ -77,8 +77,8 @@ public class ConsumerStatsImpl implements ConsumerStats { /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ public boolean blockedConsumerOnUnackedMsgs; - /** The last sent position of the cursor when the consumer joining. */ - public String lastSentPositionWhenJoining; + /** The read position of the cursor when the consumer joining. */ + public String readPositionWhenJoining; /** Address of this consumer. */ private String address; @@ -113,7 +113,7 @@ public ConsumerStatsImpl add(ConsumerStatsImpl stats) { this.availablePermits += stats.availablePermits; this.unackedMessages += stats.unackedMessages; this.blockedConsumerOnUnackedMsgs = stats.blockedConsumerOnUnackedMsgs; - this.lastSentPositionWhenJoining = stats.lastSentPositionWhenJoining; + this.readPositionWhenJoining = stats.readPositionWhenJoining; return this; } @@ -141,8 +141,8 @@ public void setClientVersion(String clientVersion) { this.clientVersion = clientVersion; } - public String getLastSentPositionWhenJoining() { - return lastSentPositionWhenJoining; + public String getReadPositionWhenJoining() { + return readPositionWhenJoining; } public String getLastAckedTime() { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index 4206a4aa8d61b..12734a5586cef 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.ArrayList; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -125,8 +126,8 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ public String keySharedMode; - /** The last sent position of the cursor. This is for Key_Shared subscription. */ - public String lastSentPosition; + /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ + public Map consumersAfterMarkDeletePosition; /** The number of non-contiguous deleted messages ranges. */ public int nonContiguousDeletedMessagesRanges; @@ -153,6 +154,7 @@ public class SubscriptionStatsImpl implements SubscriptionStats { public SubscriptionStatsImpl() { this.consumers = new ArrayList<>(); + this.consumersAfterMarkDeletePosition = new LinkedHashMap<>(); this.subscriptionProperties = new HashMap<>(); this.bucketDelayedIndexStats = new HashMap<>(); } @@ -177,6 +179,7 @@ public void reset() { lastExpireTimestamp = 0L; lastMarkDeleteAdvancedTimestamp = 0L; consumers.clear(); + consumersAfterMarkDeletePosition.clear(); nonContiguousDeletedMessagesRanges = 0; nonContiguousDeletedMessagesRangesSerializedSize = 0; earliestMsgPublishTimeInBacklog = 0L; @@ -222,6 +225,7 @@ public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { } } this.allowOutOfOrderDelivery |= stats.allowOutOfOrderDelivery; + this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; if (this.earliestMsgPublishTimeInBacklog != 0 && stats.earliestMsgPublishTimeInBacklog != 0) { From 1d83a534cf1874113eafe1c0ede301996279a3bb Mon Sep 17 00:00:00 2001 From: Jiawen Wang <74594733+summeriiii@users.noreply.github.com> Date: Wed, 9 Oct 2024 21:15:25 +0800 Subject: [PATCH 056/327] [fix][doc] Fix some typos in pip (#23288) --- pip/pip-307.md | 2 +- pip/pip-324-Alpine Docker images.md | 4 ++-- pip/pip-337.md | 6 +++--- pip/pip-352.md | 2 +- pip/pip-359.md | 4 ++-- pip/pip-368.md | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/pip/pip-307.md b/pip/pip-307.md index a919991d08991..51ab77ba29595 100644 --- a/pip/pip-307.md +++ b/pip/pip-307.md @@ -80,7 +80,7 @@ sequenceDiagram Leader Broker ->> Owner Broker: "state:Releasing:" close topic Owner Broker ->> Owner Broker: close broker topic sessions Owner Broker ->> Clients: close producers and consumers - Clients ->> Clients: reconnecting (inital delay 100ms) + Clients ->> Clients: reconnecting (initial delay 100ms) Owner Broker ->> New Owner Broker: "state:Assign:" assign new ownership New Owner Broker ->> Owner Broker: "state:Owned:" ack new ownership Clients ->> Owner Broker: lookup diff --git a/pip/pip-324-Alpine Docker images.md b/pip/pip-324-Alpine Docker images.md index c7fcc1903a93d..8d74aa41c8857 100644 --- a/pip/pip-324-Alpine Docker images.md +++ b/pip/pip-324-Alpine Docker images.md @@ -6,8 +6,8 @@ Pulsar Docker images are currently based on Ubuntu base images. While these images has served us well in the past years, there are few shortcomings. -Alpine Linux is a Linux distribution designed explicitely to work well in container environments and has strong -focus on security and a minimalistic set of included depedendencies. +Alpine Linux is a Linux distribution designed explicitly to work well in container environments and has strong +focus on security and a minimalistic set of included dependencies. ### Size of the image diff --git a/pip/pip-337.md b/pip/pip-337.md index 283bb9710de84..a50130623d758 100644 --- a/pip/pip-337.md +++ b/pip/pip-337.md @@ -184,7 +184,7 @@ public interface PulsarSslFactory extends AutoCloseable { /* * Returns the internally stored ssl context - * @throws IllegalStateException If the SSL Context has not be created before this call, then it wil throw this + * @throws IllegalStateException If the SSL Context has not be created before this call, then it will throw this * exception. */ SSLContext getInternalSslContext(); @@ -214,7 +214,7 @@ public class DefaultPulsarSslFactory implements PulsarSslFactory { } ``` -### Pulsar Commmon Changes +### Pulsar Common Changes 4 new configurations will need to be added into the Configurations like `ServiceConfiguration`, `ClientConfigurationData`, `ProxyConfiguration`, etc. All of the below will be optional. It will use the default values @@ -360,7 +360,7 @@ the DefaultAsyncHttpClient. This pattern will be common across all HTTP Clients ### Configuration -Same as [Broker Common Changes](#pulsar-commmon-changes) +Same as [Broker Common Changes](#pulsar-common-changes) ### CLI CLI tools like `PulsarClientTool` and `PulsarAdminTool` will need to be modified to support the new configurations. diff --git a/pip/pip-352.md b/pip/pip-352.md index 31641e7e1e1b5..2e43991a44c04 100644 --- a/pip/pip-352.md +++ b/pip/pip-352.md @@ -29,7 +29,7 @@ the Pulsar topic in a different order than intended. Implementing event time-based checks could mitigate this inconvenience. # Goals -* No impact on current topic compation behavior +* No impact on current topic compaction behavior * Preserve the order of messages during compaction regardless of network latencies ## In Scope diff --git a/pip/pip-359.md b/pip/pip-359.md index 52a76193d6cf2..3c7425dd77308 100644 --- a/pip/pip-359.md +++ b/pip/pip-359.md @@ -4,7 +4,7 @@ Implementation PR: [#22861](https://github.com/apache/pulsar/pull/22861) # Background knowledge In the current Pulsar client versions, from the user's perspective, when using a Pulsar Consumer, we have two main options to consume messages: -1. Pull mode, by calling `consumer.recieve()`(or `consumer.recieveAsync()`) +1. Pull mode, by calling `consumer.receive()`(or `consumer.receiveAsync()`) ```java public class ConsumerExample { public static void main(String[] args) throws PulsarClientException { @@ -25,7 +25,7 @@ public class ConsumerExample { ``` 2. Push mode, by registering a `MessageListener` interface, when building the Consumer. -When this method is used, we can't also use `consumer.receive()`(or `consumer.recieveAsync()`). +When this method is used, we can't also use `consumer.receive()`(or `consumer.receiveAsync()`). In the push mode, the MessageListener instance is called by the consumer, hence it is doing that with a thread taken from its own internal `ExecutorService` (i.e. thread pool). The problem comes when we build and use multiple Consumers from the same PulsarClient. It diff --git a/pip/pip-368.md b/pip/pip-368.md index 06bba2c12761c..f22c5b6c26cb3 100644 --- a/pip/pip-368.md +++ b/pip/pip-368.md @@ -125,7 +125,7 @@ message CommandLookupTopic { } ``` -When the client lookups a topic, it will set the client `lookupPorperties` to the `CommandLookupTopic.properties`. +When the client lookups a topic, it will set the client `lookupProperties` to the `CommandLookupTopic.properties`. ### Public API From ed01b0e14646fb4dc4e6f026059151e2d8b4d0f3 Mon Sep 17 00:00:00 2001 From: Jiawen Wang <74594733+summeriiii@users.noreply.github.com> Date: Wed, 9 Oct 2024 22:17:27 +0800 Subject: [PATCH 057/327] [fix][ml][PIP-327] fix recover from ledger when ledgerForceRecovery is true (#23426) --- .../mledger/impl/ManagedCursorImpl.java | 8 +++--- .../mledger/impl/ManagedCursorTest.java | 28 +++++++++++++++++++ 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f469b88cae8e6..7c0d13108b1c4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -549,10 +549,10 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (log.isInfoEnabled()) { log.info("[{}] Opened ledger {} for cursor {}. rc={}", ledger.getName(), ledgerId, name, rc); } - if (isBkErrorNotRecoverable(rc) || ledgerForceRecovery) { + if (isBkErrorNotRecoverable(rc) || (rc != BKException.Code.OK && ledgerForceRecovery)) { log.error("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); - // Rewind to oldest entry available + // Rewind to the oldest entry available initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); return; } else if (rc != BKException.Code.OK) { @@ -577,10 +577,10 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (log.isDebugEnabled()) { log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed()); } - if (isBkErrorNotRecoverable(rc1) || ledgerForceRecovery) { + if (isBkErrorNotRecoverable(rc1) || (rc1 != BKException.Code.OK && ledgerForceRecovery)) { log.error("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc1)); - // Rewind to oldest entry available + // Rewind to the oldest entry available initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); return; } else if (rc1 != BKException.Code.OK) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 8ae5a04a507b1..587f87a7d1d38 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -1255,6 +1255,34 @@ void cursorPersistence() throws Exception { assertEquals(c2.getMarkDeletedPosition(), p2); } + @Test(timeOut = 20000) + public void cursorPersistenceWithLedgerForceRecovery() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setLedgerForceRecovery(true); + + ManagedLedger ledger = factory.open("my_test_ledger", config); + ManagedCursor c1 = ledger.openCursor("c1"); + ledger.addEntry("dummy-entry-1".getBytes(Encoding)); + ledger.addEntry("dummy-entry-2".getBytes(Encoding)); + ledger.addEntry("dummy-entry-3".getBytes(Encoding)); + + List entries = c1.readEntries(2); + Position p1 = entries.get(1).getPosition(); + c1.markDelete(p1); + entries.forEach(Entry::release); + + entries = c1.readEntries(1); + entries.forEach(Entry::release); + + // Reopen + @Cleanup("shutdown") + ManagedLedgerFactory factory2 = new ManagedLedgerFactoryImpl(metadataStore, bkc); + ledger = factory2.open("my_test_ledger", config); + c1 = ledger.openCursor("c1"); + + assertEquals(c1.getMarkDeletedPosition(), p1); + } + @Test(timeOut = 20000) void cursorPersistence2() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", From c891eaa38a02dbbdb855b5d5d1de609f1ceb6132 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 9 Oct 2024 22:46:47 +0800 Subject: [PATCH 058/327] [fix][test] Fix flaky GetPartitionMetadataMultiBrokerTest.testCompatibilityDifferentBrokersForNonPersistentTopic (#23259) --- .../broker/admin/GetPartitionMetadataMultiBrokerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java index 60691203e777d..d1eeabdb3d7cc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java @@ -256,8 +256,8 @@ public void testCompatibilityDifferentBrokersForNonPersistentTopic(boolean confi // Initialize the connections of internal Pulsar Client. PulsarClientImpl client1 = (PulsarClientImpl) pulsar1.getClient(); PulsarClientImpl client2 = (PulsarClientImpl) pulsar2.getClient(); - client1.getLookup(pulsar2.getBrokerServiceUrl()).getBroker(TopicName.get(DEFAULT_NS + "/tp1")); - client2.getLookup(pulsar1.getBrokerServiceUrl()).getBroker(TopicName.get(DEFAULT_NS + "/tp1")); + client1.getLookup(pulsar2.getBrokerServiceUrl()).getBroker(TopicName.get(DEFAULT_NS + "/tp1")).join(); + client2.getLookup(pulsar1.getBrokerServiceUrl()).getBroker(TopicName.get(DEFAULT_NS + "/tp1")).join(); // Inject a not support flag into the connections initialized. Field field = ClientCnx.class.getDeclaredField("supportsGetPartitionedMetadataWithoutAutoCreation"); From b051dcd71d1a032353950705a1c52b53117adcd2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 9 Oct 2024 20:12:02 +0300 Subject: [PATCH 059/327] [fix] Reapply shell script parameter passthrough fix #22867 reverted in #22921 (#22923) --- bin/bookkeeper | 12 ++++++------ bin/pulsar | 38 +++++++++++++++++++------------------- 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index 668c5d4db70a8..ac7b622ac2363 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -232,20 +232,20 @@ OPTS="$OPTS $BK_METADATA_OPTIONS" #Change to BK_HOME to support relative paths cd "$BK_HOME" if [ $COMMAND == "bookie" ]; then - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.server.Main --conf $BOOKIE_CONF $@ + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.server.Main --conf $BOOKIE_CONF "$@" elif [ $COMMAND == "autorecovery" ]; then - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.replication.AutoRecoveryMain --conf $BOOKIE_CONF $@ + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.replication.AutoRecoveryMain --conf $BOOKIE_CONF "$@" elif [ $COMMAND == "localbookie" ]; then NUMBER=$1 shift - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.util.LocalBookKeeper $NUMBER $BOOKIE_CONF $@ + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.util.LocalBookKeeper $NUMBER $BOOKIE_CONF "$@" elif [ $COMMAND == "upgrade" ]; then - exec $JAVA $OPTS org.apache.bookkeeper.bookie.FileSystemUpgrade --conf $BOOKIE_CONF $@ + exec $JAVA $OPTS org.apache.bookkeeper.bookie.FileSystemUpgrade --conf $BOOKIE_CONF "$@" elif [ $COMMAND == "shell" ]; then ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}" - exec $JAVA $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF $@ + exec $JAVA $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF "$@" elif [ $COMMAND == "help" -o $COMMAND == "--help" -o $COMMAND == "-h" ]; then bookkeeper_help; else - exec $JAVA $OPTS $COMMAND $@ + exec $JAVA $OPTS $COMMAND "$@" fi diff --git a/bin/pulsar b/bin/pulsar index 09be2ac50e279..0125dd81e8bb4 100755 --- a/bin/pulsar +++ b/bin/pulsar @@ -346,56 +346,56 @@ fi cd "$PULSAR_HOME" if [ $COMMAND == "broker" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-broker.log"} - exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.PulsarBrokerStarter --broker-conf $PULSAR_BROKER_CONF $@ + exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.PulsarBrokerStarter --broker-conf $PULSAR_BROKER_CONF "$@" elif [ $COMMAND == "bookie" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"bookkeeper.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.server.Main --conf $PULSAR_BOOKKEEPER_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.server.Main --conf $PULSAR_BOOKKEEPER_CONF "$@" elif [ $COMMAND == "zookeeper" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"zookeeper.log"} - exec $JAVA ${ZK_OPTS} $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_ZK_CONF $@ + exec $JAVA ${ZK_OPTS} $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_ZK_CONF "$@" elif [ $COMMAND == "global-zookeeper" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"global-zookeeper.log"} # Allow global ZK to turn into read-only mode when it cannot reach the quorum OPTS="${OPTS} ${ZK_OPTS} -Dreadonlymode.enabled=true" - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_GLOBAL_ZK_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_GLOBAL_ZK_CONF "$@" elif [ $COMMAND == "configuration-store" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"configuration-store.log"} # Allow global ZK to turn into read-only mode when it cannot reach the quorum OPTS="${OPTS} ${ZK_OPTS} -Dreadonlymode.enabled=true" - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_CONFIGURATION_STORE_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_CONFIGURATION_STORE_CONF "$@" elif [ $COMMAND == "proxy" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-proxy.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.proxy.server.ProxyServiceStarter --config $PULSAR_PROXY_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.proxy.server.ProxyServiceStarter --config $PULSAR_PROXY_CONF "$@" elif [ $COMMAND == "websocket" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-websocket.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.websocket.service.WebSocketServiceStarter $PULSAR_WEBSOCKET_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.websocket.service.WebSocketServiceStarter $PULSAR_WEBSOCKET_CONF "$@" elif [ $COMMAND == "functions-worker" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-functions-worker.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.functions.worker.FunctionWorkerStarter -c $PULSAR_WORKER_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.functions.worker.FunctionWorkerStarter -c $PULSAR_WORKER_CONF "$@" elif [ $COMMAND == "standalone" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-standalone.log"} - exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS ${ZK_OPTS} -Dpulsar.log.file=$PULSAR_LOG_FILE -Dpulsar.config.file=$PULSAR_STANDALONE_CONF org.apache.pulsar.PulsarStandaloneStarter $@ + exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS ${ZK_OPTS} -Dpulsar.log.file=$PULSAR_LOG_FILE -Dpulsar.config.file=$PULSAR_STANDALONE_CONF org.apache.pulsar.PulsarStandaloneStarter "$@" elif [ ${COMMAND} == "autorecovery" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-autorecovery.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.replication.AutoRecoveryMain --conf $PULSAR_BOOKKEEPER_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.replication.AutoRecoveryMain --conf $PULSAR_BOOKKEEPER_CONF "$@" elif [ $COMMAND == "initialize-cluster-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataSetup $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataSetup "$@" elif [ $COMMAND == "delete-cluster-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataTeardown $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataTeardown "$@" elif [ $COMMAND == "initialize-transaction-coordinator-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarTransactionCoordinatorMetadataSetup $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarTransactionCoordinatorMetadataSetup "$@" elif [ $COMMAND == "initialize-namespace" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarInitialNamespaceSetup $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarInitialNamespaceSetup "$@" elif [ $COMMAND == "zookeeper-shell" ]; then - exec $JAVA $OPTS org.apache.zookeeper.ZooKeeperMain $@ + exec $JAVA $OPTS org.apache.zookeeper.ZooKeeperMain "$@" elif [ $COMMAND == "broker-tool" ]; then - exec $JAVA $OPTS org.apache.pulsar.broker.tools.BrokerTool $@ + exec $JAVA $OPTS org.apache.pulsar.broker.tools.BrokerTool "$@" elif [ $COMMAND == "compact-topic" ]; then - exec $JAVA $OPTS org.apache.pulsar.compaction.CompactorTool --broker-conf $PULSAR_BROKER_CONF $@ + exec $JAVA $OPTS org.apache.pulsar.compaction.CompactorTool --broker-conf $PULSAR_BROKER_CONF "$@" elif [ $COMMAND == "tokens" ]; then - exec $JAVA $OPTS org.apache.pulsar.utils.auth.tokens.TokensCliUtils $@ + exec $JAVA $OPTS org.apache.pulsar.utils.auth.tokens.TokensCliUtils "$@" elif [ $COMMAND == "version" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarVersionStarter $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarVersionStarter "$@" elif [ $COMMAND == "help" -o $COMMAND == "--help" -o $COMMAND == "-h" ]; then pulsar_help; else From 3dc0adec208100f1cc95865a0ae51fbb4e9dd360 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 10 Oct 2024 09:37:04 +0300 Subject: [PATCH 060/327] [fix][ci] Pin aquasecurity/trivy-action@0.26.0 since master is broken (#23431) --- .github/workflows/pulsar-ci.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 47a39bef9c908..bf44c51b6ad02 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -894,7 +894,7 @@ jobs: - name: Run Trivy container scan id: trivy_scan - uses: aquasecurity/trivy-action@master + uses: aquasecurity/trivy-action@0.26.0 if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} continue-on-error: true with: From acac72ea03f7c38cab99ec011b309d5e6bb4fe9d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 10 Oct 2024 12:46:42 +0300 Subject: [PATCH 061/327] [improve][broker][PIP-379] Add observability stats for "draining hashes" (#23429) --- .../pulsar/broker/service/Consumer.java | 7 + .../broker/service/DrainingHashesTracker.java | 112 +++++++ ...tStickyKeyDispatcherMultipleConsumers.java | 6 +- .../persistent/PersistentSubscription.java | 20 +- .../persistent/RescheduleReadHandler.java | 14 + .../apache/pulsar/broker/BrokerTestUtil.java | 78 ++++- .../stats/AuthenticatedConsumerStatsTest.java | 57 +--- .../broker/stats/ConsumerStatsTest.java | 276 ++++++++++++++++-- pulsar-broker/src/test/resources/log4j2.xml | 11 + .../common/policies/data/ConsumerStats.java | 48 ++- .../common/policies/data/DrainingHash.java | 41 +++ .../policies/data/SubscriptionStats.java | 24 ++ .../data/stats/ConsumerStatsImpl.java | 43 ++- .../policies/data/stats/DrainingHashImpl.java | 46 +++ .../data/stats/SubscriptionStatsImpl.java | 22 ++ .../common/util/ObjectMapperFactory.java | 3 + 16 files changed, 734 insertions(+), 74 deletions(-) create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DrainingHash.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/DrainingHashImpl.java 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 d25ebd0839df1..bcd29d86490cf 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 @@ -174,6 +174,10 @@ public class Consumer { @Setter private volatile PendingAcksMap.PendingAcksRemoveHandler pendingAcksRemoveHandler; + @Getter + @Setter + private volatile java.util.function.BiConsumer drainingHashesConsumerStatsUpdater; + public Consumer(Subscription subscription, SubType subType, String topicName, long consumerId, int priorityLevel, String consumerName, boolean isDurable, TransportCnx cnx, String appId, @@ -976,6 +980,9 @@ public ConsumerStatsImpl getStats() { if (readPositionWhenJoining != null) { stats.readPositionWhenJoining = readPositionWhenJoining.toString(); } + if (drainingHashesConsumerStatsUpdater != null) { + drainingHashesConsumerStatsUpdater.accept(this, stats); + } return stats; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java index 3521fa197a13d..46762c844db6c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java @@ -20,8 +20,18 @@ import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.PrimitiveIterator; +import java.util.concurrent.ConcurrentHashMap; import lombok.ToString; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.common.policies.data.DrainingHash; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.DrainingHashImpl; +import org.roaringbitmap.RoaringBitmap; /** * A thread-safe map to store draining hashes in the consumer. @@ -34,6 +44,8 @@ public class DrainingHashesTracker { private final Int2ObjectOpenHashMap drainingHashes = new Int2ObjectOpenHashMap<>(); int batchLevel; boolean unblockedWhileBatching; + private final Map consumerDrainingHashesStatsMap = + new ConcurrentHashMap<>(); /** * Represents an entry in the draining hashes tracker. @@ -98,6 +110,52 @@ boolean isBlocking() { } } + private class ConsumerDrainingHashesStats { + private final RoaringBitmap drainingHashes = new RoaringBitmap(); + long drainingHashesClearedTotal; + + public synchronized void addHash(int stickyHash) { + drainingHashes.add(stickyHash); + } + + public synchronized boolean clearHash(int hash) { + drainingHashes.remove(hash); + drainingHashesClearedTotal++; + boolean empty = drainingHashes.isEmpty(); + if (log.isDebugEnabled()) { + log.debug("[{}] Cleared hash {} in stats. empty={} totalCleared={} hashes={}", + dispatcherName, hash, empty, drainingHashesClearedTotal, drainingHashes.getCardinality()); + } + return empty; + } + + public synchronized void updateConsumerStats(Consumer consumer, ConsumerStatsImpl consumerStats) { + int drainingHashesUnackedMessages = 0; + List drainingHashesStats = new ArrayList<>(); + PrimitiveIterator.OfInt hashIterator = drainingHashes.stream().iterator(); + while (hashIterator.hasNext()) { + int hash = hashIterator.nextInt(); + DrainingHashEntry entry = getEntry(hash); + if (entry == null) { + log.warn("[{}] Draining hash {} not found in the tracker for consumer {}", dispatcherName, hash, + consumer); + continue; + } + int unackedMessages = entry.refCount; + DrainingHashImpl drainingHash = new DrainingHashImpl(); + drainingHash.hash = hash; + drainingHash.unackMsgs = unackedMessages; + drainingHash.blockedAttempts = entry.blockedCount; + drainingHashesStats.add(drainingHash); + drainingHashesUnackedMessages += unackedMessages; + } + consumerStats.drainingHashesCount = drainingHashesStats.size(); + consumerStats.drainingHashesClearedTotal = drainingHashesClearedTotal; + consumerStats.drainingHashesUnackedMessages = drainingHashesUnackedMessages; + consumerStats.drainingHashes = drainingHashesStats; + } + } + /** * Interface for handling the unblocking of sticky key hashes. */ @@ -127,13 +185,25 @@ public synchronized void addEntry(Consumer consumer, int stickyHash) { } DrainingHashEntry entry = drainingHashes.get(stickyHash); if (entry == null) { + if (log.isDebugEnabled()) { + log.debug("[{}] Adding and incrementing draining hash {} for consumer id:{} name:{}", dispatcherName, + stickyHash, consumer.consumerId(), consumer.consumerName()); + } entry = new DrainingHashEntry(consumer); drainingHashes.put(stickyHash, entry); + // update the consumer specific stats + consumerDrainingHashesStatsMap.computeIfAbsent(new ConsumerIdentityWrapper(consumer), + k -> new ConsumerDrainingHashesStats()).addHash(stickyHash); } else if (entry.getConsumer() != consumer) { throw new IllegalStateException( "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer + "."); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Draining hash {} incrementing {} consumer id:{} name:{}", dispatcherName, stickyHash, + entry.refCount + 1, consumer.consumerId(), consumer.consumerName()); + } } entry.incrementRefCount(); } @@ -178,7 +248,17 @@ public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boole + "."); } if (entry.decrementRefCount()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Draining hash {} removing consumer id:{} name:{}", dispatcherName, stickyHash, + consumer.consumerId(), consumer.consumerName()); + } DrainingHashEntry removed = drainingHashes.remove(stickyHash); + // update the consumer specific stats + ConsumerDrainingHashesStats drainingHashesStats = + consumerDrainingHashesStatsMap.get(new ConsumerIdentityWrapper(consumer)); + if (drainingHashesStats != null) { + drainingHashesStats.clearHash(stickyHash); + } if (!closing && removed.isBlocking()) { if (batchLevel > 0) { unblockedWhileBatching = true; @@ -186,6 +266,11 @@ public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boole unblockingHandler.stickyKeyHashUnblocked(stickyHash); } } + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Draining hash {} decrementing {} consumer id:{} name:{}", dispatcherName, stickyHash, + entry.refCount, consumer.consumerId(), consumer.consumerName()); + } } } @@ -237,5 +322,32 @@ public synchronized DrainingHashEntry getEntry(int stickyKeyHash) { */ public synchronized void clear() { drainingHashes.clear(); + consumerDrainingHashesStatsMap.clear(); + } + + /** + * Update the consumer specific stats to the target {@link ConsumerStatsImpl}. + * + * @param consumer the consumer + * @param consumerStats the consumer stats to update the values to + */ + public void updateConsumerStats(Consumer consumer, ConsumerStatsImpl consumerStats) { + consumerStats.drainingHashesCount = 0; + consumerStats.drainingHashesClearedTotal = 0; + consumerStats.drainingHashesUnackedMessages = 0; + consumerStats.drainingHashes = Collections.emptyList(); + ConsumerDrainingHashesStats consumerDrainingHashesStats = + consumerDrainingHashesStatsMap.get(new ConsumerIdentityWrapper(consumer)); + if (consumerDrainingHashesStats != null) { + consumerDrainingHashesStats.updateConsumerStats(consumer, consumerStats); + } + } + + /** + * Remove the consumer specific stats from the draining hashes tracker. + * @param consumer the consumer + */ + public void consumerRemoved(Consumer consumer) { + consumerDrainingHashesStatsMap.remove(new ConsumerIdentityWrapper(consumer)); } } \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index df053e6d8a549..1a3e2f706cba8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -157,6 +157,7 @@ public void endBatch() { drainingHashesTracker.endBatch(); } }); + consumer.setDrainingHashesConsumerStatsUpdater(drainingHashesTracker::updateConsumerStats); registerDrainingHashes(consumer, impactedConsumers.orElseThrow()); } }).exceptionally(ex -> { @@ -193,6 +194,7 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE // consumer to another. This will handle the case where a hash gets switched from an existing // consumer to another existing consumer during removal. registerDrainingHashes(consumer, impactedConsumers.orElseThrow()); + drainingHashesTracker.consumerRemoved(consumer); } } @@ -349,8 +351,8 @@ private boolean handleAddingPendingAck(Consumer consumer, long ledgerId, long en return false; } if (log.isDebugEnabled()) { - log.debug("[{}] Adding {}:{} to pending acks for consumer {} with sticky key hash {}", - getName(), ledgerId, entryId, consumer, stickyKeyHash); + log.debug("[{}] Adding {}:{} to pending acks for consumer id:{} name:{} with sticky key hash {}", + getName(), ledgerId, entryId, consumer.consumerId(), consumer.consumerName(), stickyKeyHash); } // allow adding the message to pending acks and sending the message to the consumer return true; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index eaa147b81b126..df1c23cbbcb30 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -1253,11 +1253,23 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge subStats.lastConsumedTimestamp = Math.max(subStats.lastConsumedTimestamp, consumerStats.lastConsumedTimestamp); subStats.lastAckedTimestamp = Math.max(subStats.lastAckedTimestamp, consumerStats.lastAckedTimestamp); - if (consumerKeyHashRanges != null && consumerKeyHashRanges.containsKey(consumer)) { - consumerStats.keyHashRanges = consumerKeyHashRanges.get(consumer).stream() - .map(Range::toString) - .collect(Collectors.toList()); + List keyRanges = consumerKeyHashRanges != null ? consumerKeyHashRanges.get(consumer) : null; + if (keyRanges != null) { + if (((StickyKeyDispatcher) dispatcher).isClassic()) { + // Use string representation for classic mode + consumerStats.keyHashRanges = keyRanges.stream() + .map(Range::toString) + .collect(Collectors.toList()); + } else { + // Use array representation for PIP-379 stats + consumerStats.keyHashRangeArrays = keyRanges.stream() + .map(range -> new int[]{range.getStart(), range.getEnd()}) + .collect(Collectors.toList()); + } } + subStats.drainingHashesCount += consumerStats.drainingHashesCount; + subStats.drainingHashesClearedTotal += consumerStats.drainingHashesClearedTotal; + subStats.drainingHashesUnackedMessages += consumerStats.drainingHashesUnackedMessages; }); subStats.filterProcessedMsgCount = dispatcher.getFilterProcessedMsgCount(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java index 3554f29255227..4812be58cdc78 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.BooleanSupplier; import java.util.function.LongSupplier; +import lombok.extern.slf4j.Slf4j; /** * Reschedules reads so that the possible pending read is cancelled if it's waiting for more entries. @@ -30,6 +31,7 @@ * that should be handled. This will also batch multiple calls together to reduce the number of * operations. */ +@Slf4j class RescheduleReadHandler { private static final int UNSET = -1; private static final int NO_PENDING_READ = 0; @@ -70,15 +72,27 @@ public void rescheduleRead() { // are entries in the replay queue. if (maxReadOpCount != NO_PENDING_READ && readOpCounterSupplier.getAsLong() == maxReadOpCount && hasEntriesInReplayQueue.getAsBoolean()) { + if (log.isDebugEnabled()) { + log.debug("Cancelling pending read request because it's waiting for more entries"); + } cancelPendingRead.run(); } // Re-schedule read immediately, or join the next scheduled read + if (log.isDebugEnabled()) { + log.debug("Triggering read"); + } rescheduleReadImmediately.run(); }; long rescheduleDelay = readIntervalMsSupplier.getAsLong(); if (rescheduleDelay > 0) { + if (log.isDebugEnabled()) { + log.debug("Scheduling after {} ms", rescheduleDelay); + } executor.schedule(runnable, rescheduleDelay, TimeUnit.MILLISECONDS); } else { + if (log.isDebugEnabled()) { + log.debug("Running immediately"); + } runnable.run(); } } else { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index 7ed4542b2505f..6a41e86f8934e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -21,10 +21,15 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; +import java.io.BufferedReader; import java.io.IOException; +import java.io.InputStreamReader; import java.io.StringWriter; import java.io.UncheckedIOException; +import java.net.HttpURLConnection; +import java.net.URL; import java.time.Duration; import java.util.Arrays; import java.util.UUID; @@ -37,6 +42,7 @@ import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.stream.Stream; +import lombok.SneakyThrows; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -46,7 +52,6 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.mockito.Mockito; import org.slf4j.Logger; - /** * Holds util methods used in test. */ @@ -136,6 +141,77 @@ public static void logTopicStats(Logger logger, PulsarAdmin pulsarAdmin, String } } + /** + * Logs the topic stats and internal stats for the given topic + * @param logger logger to use + * @param baseUrl Pulsar service URL + * @param topic topic name + */ + public static void logTopicStats(Logger logger, String baseUrl, String topic) { + logTopicStats(logger, baseUrl, "public", "default", topic); + } + + /** + * Logs the topic stats and internal stats for the given topic + * @param logger logger to use + * @param baseUrl Pulsar service URL + * @param tenant tenant name + * @param namespace namespace name + * @param topic topic name + */ + public static void logTopicStats(Logger logger, String baseUrl, String tenant, String namespace, String topic) { + String topicStatsUri = + String.format("%s/admin/v2/persistent/%s/%s/%s/stats", baseUrl, tenant, namespace, topic); + logger.info("[{}] stats: {}", topic, jsonPrettyPrint(getJsonResourceAsString(topicStatsUri))); + String topicStatsInternalUri = + String.format("%s/admin/v2/persistent/%s/%s/%s/internalStats", baseUrl, tenant, namespace, topic); + logger.info("[{}] internalStats: {}", topic, jsonPrettyPrint(getJsonResourceAsString(topicStatsInternalUri))); + } + + /** + * Pretty print the given JSON string + * @param jsonString JSON string to pretty print + * @return pretty printed JSON string + */ + public static String jsonPrettyPrint(String jsonString) { + try { + ObjectMapper mapper = new ObjectMapper(); + Object json = mapper.readValue(jsonString, Object.class); + ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter(); + return writer.writeValueAsString(json); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + /** + * Get the resource as a string from the given URI + */ + @SneakyThrows + public static String getJsonResourceAsString(String uri) { + URL url = new URL(uri); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); + connection.setRequestProperty("Accept", "application/json"); + try { + int responseCode = connection.getResponseCode(); + if (responseCode == 200) { + try (BufferedReader in = new BufferedReader(new InputStreamReader(connection.getInputStream()))) { + String inputLine; + StringBuilder content = new StringBuilder(); + while ((inputLine = in.readLine()) != null) { + content.append(inputLine); + } + return content.toString(); + } + } else { + throw new IOException("Failed to get resource: " + uri + ", status: " + responseCode); + } + } finally { + connection.disconnect(); + } + } + /** * Receive messages concurrently from multiple consumers and handles them using the provided message handler. * The message handler should return true if it wants to continue receiving more messages, false otherwise. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java index e8cadb72e1e04..20c1c5498ce6d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java @@ -18,11 +18,19 @@ */ package org.apache.pulsar.broker.stats; -import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Sets; import io.jsonwebtoken.Jwts; import io.jsonwebtoken.SignatureAlgorithm; +import java.security.KeyPair; +import java.security.KeyPairGenerator; +import java.security.NoSuchAlgorithmException; +import java.security.PrivateKey; +import java.time.Duration; +import java.util.Base64; +import java.util.Date; +import java.util.HashSet; +import java.util.Properties; +import java.util.Set; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.api.AuthenticationFactory; @@ -37,18 +45,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.security.KeyPair; -import java.security.KeyPairGenerator; -import java.security.NoSuchAlgorithmException; -import java.security.PrivateKey; -import java.time.Duration; -import java.util.Base64; -import java.util.Date; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Properties; -import java.util.Set; - public class AuthenticatedConsumerStatsTest extends ConsumerStatsTest{ private final String ADMIN_TOKEN; private final String TOKEN_PUBLIC_KEY; @@ -115,32 +111,6 @@ protected void setup() throws Exception { @Test public void testConsumerStatsOutput() throws Exception { - Set allowedFields = Sets.newHashSet( - "msgRateOut", - "msgThroughputOut", - "bytesOutCounter", - "msgOutCounter", - "messageAckRate", - "msgRateRedeliver", - "chunkedMessageRate", - "consumerName", - "availablePermits", - "unackedMessages", - "avgMessagesPerEntry", - "blockedConsumerOnUnackedMsgs", - "readPositionWhenJoining", - "lastAckedTime", - "lastAckedTimestamp", - "lastConsumedTime", - "lastConsumedTimestamp", - "lastConsumedFlowTimestamp", - "keyHashRanges", - "metadata", - "address", - "connectedSince", - "clientVersion", - "appId"); - final String topicName = "persistent://public/default/testConsumerStatsOutput"; final String subName = "my-subscription"; @@ -154,13 +124,6 @@ public void testConsumerStatsOutput() throws Exception { ObjectMapper mapper = ObjectMapperFactory.create(); ConsumerStats consumerStats = stats.getSubscriptions() .get(subName).getConsumers().get(0); - Assert.assertTrue(consumerStats.getLastConsumedFlowTimestamp() > 0); - JsonNode node = mapper.readTree(mapper.writer().writeValueAsString(consumerStats)); - Iterator itr = node.fieldNames(); - while (itr.hasNext()) { - String field = itr.next(); - Assert.assertTrue(allowedFields.contains(field), field + " should not be exposed"); - } // assert that role is exposed Assert.assertEquals(consumerStats.getAppId(), "admin"); consumer.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index 5b2998216e8e1..59a911500e5d9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -18,9 +18,12 @@ */ package org.apache.pulsar.broker.stats; +import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.InstanceOfAssertFactories.INTEGER; import static org.mockito.Mockito.mock; import static org.testng.Assert.assertNotEquals; import static org.testng.AssertJUnit.assertEquals; @@ -31,22 +34,29 @@ import java.io.ByteArrayOutputStream; import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.Iterator; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.PrometheusMetricsTestUtil; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.PendingAcksMap; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -67,13 +77,19 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.nar.NarClassLoader; import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.DrainingHash; +import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.assertj.core.groups.Tuple; +import org.awaitility.Awaitility; import org.testng.Assert; +import org.testng.SkipException; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -218,9 +234,24 @@ public void testUpdateStatsForActiveConsumerAndSubscription() throws Exception { Assert.assertEquals(updatedStats.getBytesOutCounter(), 1280); } - @Test - public void testConsumerStatsOutput() throws Exception { - Set allowedFields = Sets.newHashSet( + @DataProvider(name = "classicAndSubscriptionType") + public Object[][] classicAndSubscriptionType() { + return new Object[][]{ + {false, SubscriptionType.Shared}, + {true, SubscriptionType.Key_Shared}, + {false, SubscriptionType.Key_Shared} + }; + } + + @Test(dataProvider = "classicAndSubscriptionType") + public void testConsumerStatsOutput(boolean classicDispatchers, SubscriptionType subscriptionType) + throws Exception { + if (this instanceof AuthenticatedConsumerStatsTest) { + throw new SkipException("Skip test for AuthenticatedConsumerStatsTest"); + } + conf.setSubscriptionSharedUseClassicPersistentImplementation(classicDispatchers); + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(classicDispatchers); + Set expectedFields = Sets.newHashSet( "msgRateOut", "msgThroughputOut", "bytesOutCounter", @@ -233,21 +264,56 @@ public void testConsumerStatsOutput() throws Exception { "unackedMessages", "avgMessagesPerEntry", "blockedConsumerOnUnackedMsgs", - "readPositionWhenJoining", "lastAckedTime", "lastAckedTimestamp", "lastConsumedTime", "lastConsumedTimestamp", "lastConsumedFlowTimestamp", - "keyHashRanges", "metadata", "address", "connectedSince", - "clientVersion"); + "clientVersion", + "drainingHashesCount", + "drainingHashesClearedTotal", + "drainingHashesUnackedMessages" + ); + if (subscriptionType == SubscriptionType.Key_Shared) { + if (classicDispatchers) { + expectedFields.addAll(List.of( + "readPositionWhenJoining", + "keyHashRanges" + )); + } else { + expectedFields.addAll(List.of( + "drainingHashes", + "keyHashRangeArrays" + )); + } + } + final String topicName = newUniqueName("persistent://my-property/my-ns/testConsumerStatsOutput"); + final String subName = "my-subscription"; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionType(subscriptionType) + .subscriptionName(subName) + .subscribe(); + + String topicStatsUri = + String.format("%s/admin/v2/%s/stats", pulsar.getWebServiceAddress(), topicName.replace("://", "/")); + String topicStatsJson = BrokerTestUtil.getJsonResourceAsString(topicStatsUri); + ObjectMapper mapper = ObjectMapperFactory.create(); + JsonNode node = mapper.readTree(topicStatsJson).get("subscriptions").get(subName).get("consumers").get(0); + assertThat(node.fieldNames()).toIterable().containsExactlyInAnyOrderElementsOf(expectedFields); + } - final String topicName = "persistent://prop/use/ns-abc/testConsumerStatsOutput"; + @Test + public void testLastConsumerFlowTimestamp() throws PulsarClientException, PulsarAdminException { + final String topicName = newUniqueName("persistent://my-property/my-ns/testLastConsumerFlowTimestamp"); final String subName = "my-subscription"; + @Cleanup Consumer consumer = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Shared) @@ -255,18 +321,9 @@ public void testConsumerStatsOutput() throws Exception { .subscribe(); TopicStats stats = admin.topics().getStats(topicName); - ObjectMapper mapper = ObjectMapperFactory.create(); ConsumerStats consumerStats = stats.getSubscriptions() .get(subName).getConsumers().get(0); Assert.assertTrue(consumerStats.getLastConsumedFlowTimestamp() > 0); - JsonNode node = mapper.readTree(mapper.writer().writeValueAsString(consumerStats)); - Iterator itr = node.fieldNames(); - while (itr.hasNext()) { - String field = itr.next(); - Assert.assertTrue(allowedFields.contains(field), field + " should not be exposed"); - } - - consumer.close(); } @@ -481,4 +538,189 @@ public void testNonPersistentTopicSharedSubscriptionUnackedMessages() throws Exc assertEquals(0, consumers.get(0).getUnackedMessages()); } + @Test + public void testKeySharedDrainingHashesConsumerStats() throws Exception { + String topic = newUniqueName("testKeySharedDrainingHashesConsumerStats"); + String subscriptionName = "sub"; + int numberOfKeys = 10; + + // Create a producer for the topic + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .enableBatching(false) + .create(); + + // Create the first consumer (c1) for the topic + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .receiverQueueSize(100) + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + + // Get the dispatcher and selector for the topic + StickyKeyDispatcher dispatcher = getDispatcher(topic, subscriptionName); + StickyKeyConsumerSelector selector = dispatcher.getSelector(); + + // Send 20 messages with keys cycling from 0 to numberOfKeys-1 + for (int i = 0; i < 20; i++) { + String key = String.valueOf(i % numberOfKeys); + int stickyKeyHash = selector.makeStickyKeyHash(key.getBytes(StandardCharsets.UTF_8)); + log.info("Sending message with value {} key {} hash {}", key, i, stickyKeyHash); + producer.newMessage() + .key(key) + .value(i) + .send(); + } + + // Wait until all the already published messages have been pre-fetched by c1 + PendingAcksMap c1PendingAcks = dispatcher.getConsumers().get(0).getPendingAcks(); + Awaitility.await().ignoreExceptions().until(() -> c1PendingAcks.size() == 20); + + // Add a new consumer (c2) for the topic + @Cleanup + Consumer c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + + // Get the subscription stats and consumer stats + SubscriptionStats subscriptionStats = admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + ConsumerStats c1Stats = subscriptionStats.getConsumers().get(0); + ConsumerStats c2Stats = subscriptionStats.getConsumers().get(1); + + Set c2HashesByStats = new HashSet<>(); + Set c2HashesByDispatcher = new HashSet<>(); + Map c1DrainingHashesExpected = new HashMap<>(); + + int expectedDrainingHashesUnackMessages = 0; + // Determine which hashes are assigned to c2 and which are draining from c1 + // run for the same keys as the sent messages + for (int i = 0; i < 20; i++) { + // use the same key as in the sent messages + String key = String.valueOf(i % numberOfKeys); + int hash = selector.makeStickyKeyHash(key.getBytes(StandardCharsets.UTF_8)); + // Validate that the hash is assigned to c2 in stats + if ("c2".equals(findConsumerNameForHash(subscriptionStats, hash))) { + c2HashesByStats.add(hash); + } + // use the selector to determine the expected draining hashes for c1 + org.apache.pulsar.broker.service.Consumer selected = selector.select(hash); + if ("c2".equals(selected.consumerName())) { + c2HashesByDispatcher.add(hash); + c1DrainingHashesExpected.compute(hash, (k, v) -> v == null ? 1 : v + 1); + expectedDrainingHashesUnackMessages++; + } + } + + // Validate that the hashes assigned to c2 match between stats and dispatcher + assertThat(c2HashesByStats).containsExactlyInAnyOrderElementsOf(c2HashesByDispatcher); + + // Validate the draining hashes for c1 + assertThat(c1Stats.getDrainingHashes()).extracting(DrainingHash::getHash) + .containsExactlyInAnyOrderElementsOf(c2HashesByStats); + assertThat(c1Stats.getDrainingHashes()).extracting(DrainingHash::getHash, DrainingHash::getUnackMsgs) + .containsExactlyInAnyOrderElementsOf(c1DrainingHashesExpected.entrySet().stream() + .map(e -> Tuple.tuple(e.getKey(), e.getValue())).toList()); + + // Validate that c2 has no draining hashes + assertThat(c2Stats.getDrainingHashes()).isEmpty(); + + // Validate counters + assertThat(c1Stats.getDrainingHashesCount()).isEqualTo(c2HashesByStats.size()); + assertThat(c1Stats.getDrainingHashesClearedTotal()).isEqualTo(0); + assertThat(c1Stats.getDrainingHashesUnackedMessages()).isEqualTo(expectedDrainingHashesUnackMessages); + assertThat(c2Stats.getDrainingHashesCount()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesClearedTotal()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesUnackedMessages()).isEqualTo(0); + + // Send another 20 messages + for (int i = 0; i < 20; i++) { + producer.newMessage() + .key(String.valueOf(i % numberOfKeys)) + .value(i) + .send(); + } + + // Validate blocked attempts for c1 + Awaitility.await().ignoreExceptions().untilAsserted(() -> { + SubscriptionStats stats = admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + assertThat(stats.getConsumers().get(0).getDrainingHashes()).isNotEmpty().allSatisfy(dh -> { + assertThat(dh).extracting(DrainingHash::getBlockedAttempts) + .asInstanceOf(INTEGER) + .isGreaterThan(0); + }); + }); + + // Acknowledge messages that were sent before c2 joined, to clear all draining hashes + for (int i = 0; i < 20; i++) { + Message message = c1.receive(1, TimeUnit.SECONDS); + log.info("Acking message with value {} key {}", message.getValue(), message.getKey()); + c1.acknowledge(message); + + if (i == 18) { + // Validate that there is one draining hash left + Awaitility.await().pollInterval(Duration.ofSeconds(1)).atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + SubscriptionStats stats = + admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + assertThat(stats.getConsumers().get(0)).satisfies(consumerStats -> { + assertThat(consumerStats) + .describedAs("Consumer stats should have one draining hash %s", consumerStats) + .extracting(ConsumerStats::getDrainingHashes) + .asList().hasSize(1); + }); + }); + } + + if (i == 19) { + // Validate that there are no draining hashes left + Awaitility.await().pollInterval(Duration.ofSeconds(1)).atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + SubscriptionStats stats = + admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + assertThat(stats.getConsumers().get(0)).satisfies(consumerStats -> { + assertThat(consumerStats).extracting(ConsumerStats::getDrainingHashes) + .asList().isEmpty(); + }); + }); + } + } + + // Get the subscription stats and consumer stats + subscriptionStats = admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + c1Stats = subscriptionStats.getConsumers().get(0); + c2Stats = subscriptionStats.getConsumers().get(1); + + // Validate counters + assertThat(c1Stats.getDrainingHashesCount()).isEqualTo(0); + assertThat(c1Stats.getDrainingHashesClearedTotal()).isEqualTo(c2HashesByStats.size()); + assertThat(c1Stats.getDrainingHashesUnackedMessages()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesCount()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesClearedTotal()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesUnackedMessages()).isEqualTo(0); + + } + + private String findConsumerNameForHash(SubscriptionStats subscriptionStats, int hash) { + return findConsumerForHash(subscriptionStats, hash).map(ConsumerStats::getConsumerName).orElse(null); + } + + private Optional findConsumerForHash(SubscriptionStats subscriptionStats, int hash) { + return subscriptionStats.getConsumers().stream() + .filter(consumerStats -> consumerStats.getKeyHashRangeArrays().stream() + .anyMatch(hashRanges -> hashRanges[0] <= hash && hashRanges[1] >= hash)) + .findFirst(); + } + + @SneakyThrows + private StickyKeyDispatcher getDispatcher(String topic, String subscription) { + return (StickyKeyDispatcher) pulsar.getBrokerService().getTopicIfExists(topic).get() + .get().getSubscription(subscription).getDispatcher(); + } } diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml index 09a89702ee2ac..a0732096f2845 100644 --- a/pulsar-broker/src/test/resources/log4j2.xml +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -36,5 +36,16 @@ + diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java index d2d3600df96ed..16dce5903f492 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java @@ -73,8 +73,41 @@ public interface ConsumerStats { boolean isBlockedConsumerOnUnackedMsgs(); /** The read position of the cursor when the consumer joining. */ + @Deprecated String getReadPositionWhenJoining(); + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the current number of hashes in the draining state for this consumer. + * + * @return the current number of hashes in the draining state for this consumer + */ + int getDrainingHashesCount(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the total number of hashes cleared from the draining state since the consumer connected. + * + * @return the total number of hashes cleared from the draining state since the consumer connected + */ + long getDrainingHashesClearedTotal(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the total number of unacked messages for all draining hashes for this consumer. + * + * @return the total number of unacked messages for all draining hashes for this consumer + */ + int getDrainingHashesUnackedMessages(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the draining hashes for this consumer. + * + * @return a list of draining hashes for this consumer + */ + List getDrainingHashes(); + /** Address of this consumer. */ String getAddress(); @@ -88,9 +121,20 @@ public interface ConsumerStats { long getLastConsumedTimestamp(); long getLastConsumedFlowTimestamp(); - /** Hash ranges assigned to this consumer if is Key_Shared sub mode. **/ + /** + * Hash ranges assigned to this consumer if in Key_Shared subscription mode. + * This format and field is used when `subscriptionKeySharedUseClassicPersistentImplementation` is set to `false` + * (default). + */ + List getKeyHashRangeArrays(); + + /** + * Hash ranges assigned to this consumer if in Key_Shared subscription mode. + * This format and field is used when `subscriptionKeySharedUseClassicPersistentImplementation` is set to `true`. + */ + @Deprecated List getKeyHashRanges(); /** Metadata (key/value strings) associated with this consumer. */ Map getMetadata(); -} +} \ No newline at end of file diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DrainingHash.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DrainingHash.java new file mode 100644 index 0000000000000..685b0b74e64b9 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DrainingHash.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.common.policies.data; + +/** + * Contains information about a draining hash in a Key_Shared subscription. + * @see ConsumerStats + */ +public interface DrainingHash { + /** + * Get the sticky key hash value of the draining hash. + * @return the sticky hash value + */ + int getHash(); + /** + * Get number of unacknowledged messages for the draining hash. + * @return number of unacknowledged messages + */ + int getUnackMsgs(); + /** + * Get the number of times the hash has blocked an attempted delivery of a message. + * @return number of times the hash has blocked an attempted delivery of a message + */ + int getBlockedAttempts(); +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index ce3a080a855da..95e7c65266bff 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -121,6 +121,30 @@ public interface SubscriptionStats { /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ Map getConsumersAfterMarkDeletePosition(); + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the current number of hashes in the draining state. + * + * @return the current number of hashes in the draining state + */ + int getDrainingHashesCount(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the total number of hashes cleared from the draining state for the connected consumers. + * + * @return the total number of hashes cleared from the draining state for the connected consumers + */ + long getDrainingHashesClearedTotal(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the total number of unacked messages for all draining hashes. + * + * @return the total number of unacked messages for all draining hashes + */ + int getDrainingHashesUnackedMessages(); + /** SubscriptionProperties (key/value strings) associated with this subscribe. */ Map getSubscriptionProperties(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java index de36b330b7f1a..8811247cb2de3 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java @@ -23,6 +23,7 @@ import java.util.Objects; import lombok.Data; import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.DrainingHash; import org.apache.pulsar.common.util.DateFormatter; /** @@ -80,6 +81,30 @@ public class ConsumerStatsImpl implements ConsumerStats { /** The read position of the cursor when the consumer joining. */ public String readPositionWhenJoining; + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The current number of hashes in the draining state. + */ + public int drainingHashesCount; + + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The total number of hashes cleared from the draining state for + * the consumer. + */ + public long drainingHashesClearedTotal; + + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The total number of unacked messages for all draining hashes. + */ + public int drainingHashesUnackedMessages; + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the draining hashes for this consumer. + * + * @return a list of draining hashes for this consumer + */ + public List drainingHashes; + /** Address of this consumer. */ private String address; /** Timestamp of connection. */ @@ -96,7 +121,17 @@ public class ConsumerStatsImpl implements ConsumerStats { public long lastConsumedFlowTimestamp; - /** Hash ranges assigned to this consumer if is Key_Shared sub mode. **/ + /** + * Hash ranges assigned to this consumer if in Key_Shared subscription mode. + * This format and field is used when `subscriptionKeySharedUseClassicPersistentImplementation` is set to `false` + * (default). + */ + public List keyHashRangeArrays; + + /** + * Hash ranges assigned to this consumer if in Key_Shared subscription mode. + * This format and field is used when `subscriptionKeySharedUseClassicPersistentImplementation` is set to `true`. + */ public List keyHashRanges; /** Metadata (key/value strings) associated with this consumer. */ @@ -114,6 +149,12 @@ public ConsumerStatsImpl add(ConsumerStatsImpl stats) { this.unackedMessages += stats.unackedMessages; this.blockedConsumerOnUnackedMsgs = stats.blockedConsumerOnUnackedMsgs; this.readPositionWhenJoining = stats.readPositionWhenJoining; + this.drainingHashesCount = stats.drainingHashesCount; + this.drainingHashesClearedTotal += stats.drainingHashesClearedTotal; + this.drainingHashesUnackedMessages = stats.drainingHashesUnackedMessages; + this.drainingHashes = stats.drainingHashes; + this.keyHashRanges = stats.keyHashRanges; + this.keyHashRangeArrays = stats.keyHashRangeArrays; return this; } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/DrainingHashImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/DrainingHashImpl.java new file mode 100644 index 0000000000000..134bdac597b7c --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/DrainingHashImpl.java @@ -0,0 +1,46 @@ +/* + * 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.common.policies.data.stats; + +import lombok.Data; +import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.DrainingHash; + +/** + * Contains information about a draining hash in a Key_Shared subscription. + * @see ConsumerStats + */ +@Data +public class DrainingHashImpl implements DrainingHash { + /** + * Get the sticky key hash value of the draining hash. + * @return the sticky hash value + */ + public int hash; + /** + * Get number of unacknowledged messages for the draining hash. + * @return number of unacknowledged messages + */ + public int unackMsgs; + /** + * Get the number of times the hash has blocked an attempted delivery of a message. + * @return number of times the hash has blocked an attempted delivery of a message + */ + public int blockedAttempts; +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index 12734a5586cef..02df9b7870023 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -129,6 +129,22 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ public Map consumersAfterMarkDeletePosition; + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The current number of hashes in the draining state. + */ + public int drainingHashesCount; + + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The total number of hashes cleared from the draining state + * for the connected consumers. + */ + public long drainingHashesClearedTotal; + + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The total number of unacked messages for all draining hashes. + */ + public int drainingHashesUnackedMessages; + /** The number of non-contiguous deleted messages ranges. */ public int nonContiguousDeletedMessagesRanges; @@ -180,6 +196,9 @@ public void reset() { lastMarkDeleteAdvancedTimestamp = 0L; consumers.clear(); consumersAfterMarkDeletePosition.clear(); + drainingHashesCount = 0; + drainingHashesClearedTotal = 0L; + drainingHashesUnackedMessages = 0; nonContiguousDeletedMessagesRanges = 0; nonContiguousDeletedMessagesRangesSerializedSize = 0; earliestMsgPublishTimeInBacklog = 0L; @@ -226,6 +245,9 @@ public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { } this.allowOutOfOrderDelivery |= stats.allowOutOfOrderDelivery; this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); + this.drainingHashesCount += stats.drainingHashesCount; + this.drainingHashesClearedTotal += stats.drainingHashesClearedTotal; + this.drainingHashesUnackedMessages += stats.drainingHashesUnackedMessages; this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; if (this.earliestMsgPublishTimeInBacklog != 0 && stats.earliestMsgPublishTimeInBacklog != 0) { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java index 7b235cfa341d1..b737d68d5ea9f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java @@ -56,6 +56,7 @@ import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; import org.apache.pulsar.common.policies.data.DispatchRate; +import org.apache.pulsar.common.policies.data.DrainingHash; import org.apache.pulsar.common.policies.data.FailureDomain; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.policies.data.FunctionInstanceStats; @@ -96,6 +97,7 @@ import org.apache.pulsar.common.policies.data.impl.DelayedDeliveryPoliciesImpl; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.DrainingHashImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentPartitionedTopicStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentPublisherStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentReplicatorStatsImpl; @@ -243,6 +245,7 @@ private static void setAnnotationsModule(ObjectMapper mapper) { resolver.addMapping(DispatchRate.class, DispatchRateImpl.class); resolver.addMapping(TopicStats.class, TopicStatsImpl.class); resolver.addMapping(ConsumerStats.class, ConsumerStatsImpl.class); + resolver.addMapping(DrainingHash.class, DrainingHashImpl.class); resolver.addMapping(NonPersistentPublisherStats.class, NonPersistentPublisherStatsImpl.class); resolver.addMapping(NonPersistentReplicatorStats.class, NonPersistentReplicatorStatsImpl.class); resolver.addMapping(NonPersistentSubscriptionStats.class, NonPersistentSubscriptionStatsImpl.class); From 9f8b4a6d15899c7255c525299e1d6011a09b940e Mon Sep 17 00:00:00 2001 From: Masahiro Sakamoto Date: Thu, 10 Oct 2024 19:03:56 +0900 Subject: [PATCH 062/327] [fix][broker] Fix ack hole in cursor for geo-replication (#20931) Co-authored-by: Masahiro Sakamoto --- .../persistent/PersistentReplicator.java | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 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 b3d7546beed81..bcb1f759540b4 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 @@ -116,6 +116,7 @@ public abstract class PersistentReplicator extends AbstractReplicator protected final ReplicatorStatsImpl stats = new ReplicatorStatsImpl(); protected volatile boolean fetchSchemaInProgress = false; + private volatile boolean waitForCursorRewinding = false; public PersistentReplicator(String localCluster, PersistentTopic localTopic, ManagedCursor cursor, String remoteCluster, String remoteTopic, @@ -143,9 +144,15 @@ public PersistentReplicator(String localCluster, PersistentTopic localTopic, Man @Override protected void setProducerAndTriggerReadEntries(Producer producer) { - // Rewind the cursor to be sure to read again all non-acked messages sent while restarting. - cursor.rewind(); - cursor.cancelPendingReadRequest(); + waitForCursorRewinding = true; + + // Repeat until there are no read operations in progress + if (STATE_UPDATER.get(this) == State.Starting && HAVE_PENDING_READ_UPDATER.get(this) == TRUE + && !cursor.cancelPendingReadRequest()) { + brokerService.getPulsar().getExecutor() + .schedule(() -> setProducerAndTriggerReadEntries(producer), 10, TimeUnit.MILLISECONDS); + return; + } /** * 1. Try change state to {@link Started}. @@ -158,6 +165,7 @@ protected void setProducerAndTriggerReadEntries(Producer producer) { if (!(producer instanceof ProducerImpl)) { log.error("[{}] The partitions count between two clusters is not the same, the replicator can not be" + " created successfully: {}", replicatorId, state); + waitForCursorRewinding = false; doCloseProducerAsync(producer, () -> {}); throw new ClassCastException(producer.getClass().getName() + " can not be cast to ProducerImpl"); } @@ -168,6 +176,11 @@ protected void setProducerAndTriggerReadEntries(Producer producer) { backOff.reset(); // activate cursor: so, entries can be cached. this.cursor.setActive(); + + // Rewind the cursor to be sure to read again all non-acked messages sent while restarting + cursor.rewind(); + waitForCursorRewinding = false; + // read entries readMoreEntries(); } else { @@ -183,6 +196,7 @@ protected void setProducerAndTriggerReadEntries(Producer producer) { log.error("[{}] Replicator state is not expected, so close the producer. Replicator state: {}", replicatorId, changeStateRes.getRight()); } + waitForCursorRewinding = false; // Close the producer if change the state fail. doCloseProducerAsync(producer, () -> {}); } @@ -296,6 +310,11 @@ protected void readMoreEntries() { // Schedule read if (HAVE_PENDING_READ_UPDATER.compareAndSet(this, FALSE, TRUE)) { + if (waitForCursorRewinding) { + log.info("[{}] Skip the reading because repl producer is starting", replicatorId); + HAVE_PENDING_READ_UPDATER.set(this, FALSE); + return; + } if (log.isDebugEnabled()) { log.debug("[{}] Schedule read of {} messages or {} bytes", replicatorId, messagesToRead, bytesToRead); From 8600316113662460fcd559bae85a1646f4862083 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 10 Oct 2024 13:18:43 +0300 Subject: [PATCH 063/327] [improve][client] Increase default Java client connectionMaxIdleSeconds to 60 seconds (#23430) --- .../apache/pulsar/client/impl/conf/ClientConfigurationData.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index c1c2e75925502..c4fbf1e1744d6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -139,7 +139,7 @@ public class ClientConfigurationData implements Serializable, Cloneable { value = "Release the connection if it is not used for more than [connectionMaxIdleSeconds] seconds. " + "If [connectionMaxIdleSeconds] < 0, disabled the feature that auto release the idle connections" ) - private int connectionMaxIdleSeconds = 25; + private int connectionMaxIdleSeconds = 60; @ApiModelProperty( name = "useTcpNoDelay", From b334c4f637bdd32787494c16e9d34169f1a25812 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Thu, 10 Oct 2024 18:37:27 +0800 Subject: [PATCH 064/327] [improve][broker] PIP-383: Support granting/revoking permissions for multiple topics (#23372) --- .../authorization/AuthorizationProvider.java | 13 ++++ .../authorization/AuthorizationService.java | 11 +++ .../PulsarAuthorizationProvider.java | 78 +++++++++++++++++++ .../pulsar/broker/admin/AdminResource.java | 11 +++ .../broker/admin/impl/NamespacesBase.java | 75 ++++++++++++++++++ .../pulsar/broker/admin/v2/Namespaces.java | 44 +++++++++++ .../pulsar/broker/admin/AdminApi2Test.java | 59 ++++++++++++++ .../admin/GrantTopicPermissionOptions.java | 36 +++++++++ .../pulsar/client/admin/Namespaces.java | 28 +++++++ .../admin/RevokeTopicPermissionOptions.java | 32 ++++++++ .../client/admin/internal/NamespacesImpl.java | 26 +++++++ 11 files changed, 413 insertions(+) create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/GrantTopicPermissionOptions.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/RevokeTopicPermissionOptions.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java index 7d25580ff92bb..ffb38f770a9cc 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java @@ -20,12 +20,15 @@ import java.io.Closeable; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.resources.PulsarResources; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; @@ -223,6 +226,16 @@ CompletableFuture revokeSubscriptionPermissionAsync(NamespaceName namespac CompletableFuture grantPermissionAsync(TopicName topicName, Set actions, String role, String authDataJson); + default CompletableFuture grantPermissionAsync(List options) { + return FutureUtil.failedFuture(new IllegalStateException( + String.format("grantPermissionAsync is not supported by the Authorization"))); + } + + default CompletableFuture revokePermissionAsync(List options) { + return FutureUtil.failedFuture(new IllegalStateException( + String.format("revokePermissionAsync is not supported by the Authorization"))); + } + /** * Revoke authorization-action permission on a topic to the given client. diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java index c121d93b9b750..2951eb1f2973f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java @@ -20,6 +20,7 @@ import static java.util.concurrent.TimeUnit.SECONDS; import java.net.SocketAddress; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -32,6 +33,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationParameters; import org.apache.pulsar.broker.resources.PulsarResources; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; @@ -181,6 +184,14 @@ public CompletableFuture grantPermissionAsync(TopicName topicName, Set grantPermissionAsync(List options) { + return provider.grantPermissionAsync(options); + } + + public CompletableFuture revokePermissionAsync(List options) { + return provider.revokePermissionAsync(options); + } + /** * Revoke authorization-action permission on a topic to the given client. * diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java index a39c3d0560760..0af63724cc812 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java @@ -24,14 +24,18 @@ import java.io.IOException; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.stream.Stream; import javax.ws.rs.core.Response; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.resources.PulsarResources; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; @@ -251,6 +255,80 @@ public CompletableFuture grantPermissionAsync(TopicName topicName, Set grantPermissionAsync(List options) { + return checkNamespace(options.stream().map(o -> TopicName.get(o.getTopic()).getNamespace())) + .thenCompose(__ -> getPoliciesReadOnlyAsync()) + .thenCompose(readonly -> { + if (readonly) { + if (log.isDebugEnabled()) { + log.debug("Policies are read-only. Broker cannot do read-write operations"); + } + throw new IllegalStateException("policies are in readonly mode"); + } + TopicName topicName = TopicName.get(options.get(0).getTopic()); + return pulsarResources.getNamespaceResources() + .setPoliciesAsync(topicName.getNamespaceObject(), policies -> { + options.stream().forEach(o -> { + final String topicUri = TopicName.get(o.getTopic()).toString(); + policies.auth_policies.getTopicAuthentication() + .computeIfAbsent(topicUri, __ -> new HashMap<>()) + .put(o.getRole(), o.getActions()); + }); + return policies; + }).whenComplete((__, ex) -> { + if (ex != null) { + log.error("Failed to grant permissions for {}", options); + } else { + log.info("Successfully granted access for {}", options); + } + }); + }); + } + + @Override + public CompletableFuture revokePermissionAsync(List options) { + return checkNamespace(options.stream().map(o -> TopicName.get(o.getTopic()).getNamespace())) + .thenCompose(__ -> getPoliciesReadOnlyAsync()) + .thenCompose(readonly -> { + if (readonly) { + if (log.isDebugEnabled()) { + log.debug("Policies are read-only. Broker cannot do read-write operations"); + } + throw new IllegalStateException("policies are in readonly mode"); + } + TopicName topicName = TopicName.get(options.get(0).getTopic()); + return pulsarResources.getNamespaceResources() + .setPoliciesAsync(topicName.getNamespaceObject(), policies -> { + options.stream().forEach(o -> { + final String topicUri = TopicName.get(o.getTopic()).toString(); + policies.auth_policies.getTopicAuthentication() + .computeIfPresent(topicUri, (topicNameUri, roles) -> { + roles.remove(o.getRole()); + if (roles.isEmpty()) { + return null; + } + return roles; + }); + }); + return policies; + }).whenComplete((__, ex) -> { + if (ex != null) { + log.error("Failed to revoke permissions for {}", options, ex); + } else { + log.info("Successfully revoke permissions for {}", options); + } + }); + }); + } + + private CompletableFuture checkNamespace(Stream namespaces) { + boolean sameNamespace = namespaces.distinct().count() == 1; + if (!sameNamespace) { + throw new IllegalArgumentException("The namespace should be the same"); + } + return CompletableFuture.completedFuture(null); + } + @Override public CompletableFuture revokePermissionAsync(TopicName topicName, String role) { return getPoliciesReadOnlyAsync().thenCompose(readonly -> { 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 3268f07b13d88..45772dc279bab 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 @@ -924,4 +924,15 @@ protected void validateOffloadPolicies(OffloadPoliciesImpl offloadPolicies) { "The bucket must be specified for namespace offload."); } } + + protected CompletableFuture internalCheckTopicExists(TopicName topicName) { + return pulsar().getNamespaceService().checkTopicExists(topicName) + .thenAccept(info -> { + boolean exists = info.isExists(); + info.recycle(); + if (!exists) { + throw new RestException(Status.NOT_FOUND, getTopicNotFoundErrorMessage(topicName.toString())); + } + }); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index 18c80d6bef4bf..d80e2487b4f1c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; import javax.annotation.Nonnull; import javax.ws.rs.WebApplicationException; import javax.ws.rs.container.AsyncResponse; @@ -65,8 +66,10 @@ import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.NamedEntity; @@ -613,6 +616,78 @@ protected CompletableFuture internalGrantPermissionOnNamespaceAsync(String }); } + protected CompletableFuture internalGrantPermissionOnTopicsAsync(List options) { + return checkNamespace(options.stream().map(o -> TopicName.get(o.getTopic()).getNamespace())) + .thenCompose(__ -> validateAdminAccessForTenantAsync( + TopicName.get(options.get(0).getTopic()).getTenant()) + ).thenCompose(__ -> internalCheckTopicExists(options.stream().map(o -> TopicName.get(o.getTopic())))) + .thenCompose(__ -> getAuthorizationService().grantPermissionAsync(options)) + .thenAccept(unused -> log.info("[{}] Successfully granted access for {}", clientAppId(), options)) + .exceptionally(ex -> { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + //The IllegalArgumentException and the IllegalStateException were historically thrown by the + // grantPermissionAsync method, so we catch them here to ensure backwards compatibility. + if (realCause instanceof MetadataStoreException.NotFoundException + || realCause instanceof IllegalArgumentException) { + log.warn("[{}] Failed to grant permissions for namespace {}: does not exist", clientAppId(), + namespaceName, ex); + throw new RestException(Status.NOT_FOUND, "Topic's namespace does not exist"); + } else if (realCause instanceof MetadataStoreException.BadVersionException + || realCause instanceof IllegalStateException) { + log.warn("[{}] Failed to grant permissions for namespace {}: {}", + clientAppId(), namespaceName, ex.getCause().getMessage(), ex); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } else { + log.error("[{}] Failed to grant permissions for namespace {}", + clientAppId(), namespaceName, ex); + throw new RestException(realCause); + } + }); + } + + protected CompletableFuture internalRevokePermissionOnTopicsAsync( + List options) { + return checkNamespace(options.stream().map(o -> TopicName.get(o.getTopic()).getNamespace())) + .thenCompose(__ -> validateAdminAccessForTenantAsync( + TopicName.get(options.get(0).getTopic()).getTenant())) + .thenCompose(__ -> internalCheckTopicExists(options.stream().map(o -> TopicName.get(o.getTopic())))) + .thenCompose(__ -> getAuthorizationService().revokePermissionAsync(options)) + .thenAccept(unused -> log.info("[{}] Successfully revoke access for {}", clientAppId(), options)) + .exceptionally(ex -> { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + //The IllegalArgumentException and the IllegalStateException were historically thrown by the + // grantPermissionAsync method, so we catch them here to ensure backwards compatibility. + if (realCause instanceof MetadataStoreException.NotFoundException + || realCause instanceof IllegalArgumentException) { + log.warn("[{}] Failed to revoke permissions for namespace {}: does not exist", clientAppId(), + namespaceName, ex); + throw new RestException(Status.NOT_FOUND, "Topic's namespace does not exist"); + } else if (realCause instanceof MetadataStoreException.BadVersionException + || realCause instanceof IllegalStateException) { + log.warn("[{}] Failed to revoke permissions for namespace {}: {}", + clientAppId(), namespaceName, ex.getCause().getMessage(), ex); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } else { + log.error("[{}] Failed to revoke permissions for namespace {}", + clientAppId(), namespaceName, ex); + throw new RestException(realCause); + } + }); + } + + private CompletableFuture checkNamespace(Stream namespaces) { + boolean sameNamespace = namespaces.distinct().count() == 1; + if (!sameNamespace) { + throw new RestException(Status.BAD_REQUEST, "The namespace should be the same"); + } + return CompletableFuture.completedFuture(null); + } + + private CompletableFuture internalCheckTopicExists(Stream topicNameStream) { + List topicNames = topicNameStream.collect(Collectors.toList()); + return CompletableFuture.allOf(topicNames.stream().map(topic -> internalCheckTopicExists(topic)) + .toArray(CompletableFuture[]::new)); + } protected CompletableFuture internalGrantPermissionOnSubscriptionAsync(String subscription, Set roles) { 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 54cceaf09e9fe..36150ee21b32c 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 @@ -50,6 +50,8 @@ import org.apache.pulsar.broker.admin.impl.NamespacesBase; import org.apache.pulsar.broker.admin.impl.OffloaderObjectsScannerUtils; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; import org.apache.pulsar.common.naming.NamespaceName; @@ -314,6 +316,48 @@ public void grantPermissionOnNamespace(@Suspended AsyncResponse asyncResponse, }); } + @POST + @Path("/grantPermissionsOnTopics") + @ApiOperation(value = "Grant new permissions to a role on multi-topics.") + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), + @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "tenant/namespace/topic doesn't exit"), + @ApiResponse(code = 500, message = "Internal server error") }) + public void grantPermissionsOnTopics(@Suspended final AsyncResponse asyncResponse, + List options) { + internalGrantPermissionOnTopicsAsync(options) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + log.error("[{}] Failed to grant permissions {}", + clientAppId(), options, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); + } + + @POST + @Path("/revokePermissionsOnTopics") + @ApiOperation(value = "Revoke new permissions to a role on multi-topics.") + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), + @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "tenant/namespace/topic doesn't exit"), + @ApiResponse(code = 500, message = "Internal server error") }) + public void revokePermissionsOnTopics(@Suspended final AsyncResponse asyncResponse, + List options) { + internalRevokePermissionOnTopicsAsync(options) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + log.error("[{}] Failed to revoke permissions {}", + clientAppId(), options, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); + } + @POST @Path("/{property}/{namespace}/permissions/subscription/{subscription}") @ApiOperation(hidden = true, value = "Grant a new permission to roles for a subscription." diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java index 900babbecf4ad..3f5ee721a7e6b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java @@ -87,11 +87,13 @@ import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; import org.apache.pulsar.broker.testcontext.MockEntryFilterProvider; import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; import org.apache.pulsar.client.admin.Mode; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.PulsarAdminException.PreconditionFailedException; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.client.admin.Topics.QueryParam; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -3685,4 +3687,61 @@ public void testIsolationPolicyUnloadsNSWithPrimaryChanged(final String topicTyp List.of(".*", "broker.*") ); } + + @Test + public void testGrantAndRevokePermissions() throws Exception { + + String namespace = newUniqueName(defaultTenant + "/") + "-unload-test-"; + String namespace2 = newUniqueName(defaultTenant + "/") + "-unload-test-"; + admin.namespaces().createNamespace(namespace, Set.of("test")); + admin.namespaces().createNamespace(namespace2, Set.of("test")); + // + final String topic1 = "persistent://" + namespace + "/test1"; + final String topic2 = "persistent://" + namespace + "/test2"; + final String topic3 = "non-persistent://" + namespace + "/test3"; + final String topic4 = "persistent://" + namespace2 + "/test4";; + + admin.topics().createPartitionedTopic(topic1, 3); + admin.topics().createPartitionedTopic(topic2, 3); + admin.topics().createPartitionedTopic(topic3, 3); + admin.topics().createPartitionedTopic(topic4, 3); + pulsarClient.newProducer().topic(topic1).create().close(); + pulsarClient.newProducer().topic(topic2).create().close(); + pulsarClient.newProducer().topic(topic3).create().close(); + pulsarClient.newProducer().topic(topic4).create().close(); + + List grantPermissionOptions = new ArrayList<>(); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic1).role("role1").actions(Set.of(AuthAction.produce)).build()); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic4).role("role4").actions(Set.of(AuthAction.produce)).build()); + try { + admin.namespaces().grantPermissionOnTopics(grantPermissionOptions); + fail("Should go here, because there are two namespaces"); + } catch (Exception ex) { + Assert.assertTrue(ex != null); + } + grantPermissionOptions.clear(); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic1).role("role1").actions(Set.of(AuthAction.produce)).build()); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic2).role("role2").actions(Set.of(AuthAction.consume)).build()); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic3).role("role3").actions(Set.of(AuthAction.produce, AuthAction.consume)).build()); + admin.namespaces().grantPermissionOnTopics(grantPermissionOptions); + + final Map> permissions1 = admin.topics().getPermissions(topic1); + final Map> permissions2 = admin.topics().getPermissions(topic2); + final Map> permissions3 = admin.topics().getPermissions(topic3); + + Assert.assertEquals(permissions1.get("role1"), Set.of(AuthAction.produce)); + Assert.assertEquals(permissions2.get("role2"), Set.of(AuthAction.consume)); + Assert.assertEquals(permissions3.get("role3"), Set.of(AuthAction.produce, AuthAction.consume)); + // + List revokePermissionOptions = new ArrayList<>(); + revokePermissionOptions.add(RevokeTopicPermissionOptions.builder().topic(topic1).role("role1").build()); + revokePermissionOptions.add(RevokeTopicPermissionOptions.builder().topic(topic2).role("role2").build()); + admin.namespaces().revokePermissionOnTopics(revokePermissionOptions); + + final Map> permissions11 = admin.topics().getPermissions(topic1); + final Map> permissions22 = admin.topics().getPermissions(topic2); + + Assert.assertTrue(permissions11.isEmpty()); + Assert.assertTrue(permissions22.isEmpty()); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/GrantTopicPermissionOptions.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/GrantTopicPermissionOptions.java new file mode 100644 index 0000000000000..e365a086a771f --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/GrantTopicPermissionOptions.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.admin; + +import java.util.Set; +import lombok.Builder; +import lombok.Data; +import org.apache.pulsar.common.policies.data.AuthAction; + +@Data +@Builder +public class GrantTopicPermissionOptions { + + private final String topic; + + private final String role; + + private final Set actions; + +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java index 65124a6a76a8f..28ad852064b4f 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java @@ -703,6 +703,34 @@ public interface Namespaces { */ CompletableFuture grantPermissionOnNamespaceAsync(String namespace, String role, Set actions); + /** + * Grant permissions on topics asynchronously. + * @param options + * @return + */ + CompletableFuture grantPermissionOnTopicsAsync(List options); + + /** + * Grant permissions on topics. + * @param options + * @throws PulsarAdminException + */ + void grantPermissionOnTopics(List options) throws PulsarAdminException; + + /** + * Revoke permissions on topics asynchronously. + * @param options + * @return + */ + CompletableFuture revokePermissionOnTopicsAsync(List options); + + /** + * Revoke permissions on topics. + * @param options + * @throws PulsarAdminException + */ + void revokePermissionOnTopics(List options) throws PulsarAdminException; + /** * Revoke permissions on a namespace. *

diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/RevokeTopicPermissionOptions.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/RevokeTopicPermissionOptions.java new file mode 100644 index 0000000000000..38e33c966b284 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/RevokeTopicPermissionOptions.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.admin; + +import lombok.Builder; +import lombok.Data; + +@Data +@Builder +public class RevokeTopicPermissionOptions { + + private final String topic; + + private final String role; + +} diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java index 7d41c7203d2c7..7695abdd4809b 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java @@ -28,9 +28,11 @@ import javax.ws.rs.client.WebTarget; import javax.ws.rs.core.MediaType; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; import org.apache.pulsar.client.admin.Namespaces; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.NamespaceName; @@ -288,6 +290,30 @@ public CompletableFuture grantPermissionOnNamespaceAsync( return asyncPostRequest(path, Entity.entity(actions, MediaType.APPLICATION_JSON)); } + @Override + public CompletableFuture grantPermissionOnTopicsAsync(List options) { + final WebTarget base = adminV2Namespaces; + WebTarget path = base.path("/grantPermissionsOnTopics"); + return asyncPostRequest(path, Entity.entity(options, MediaType.APPLICATION_JSON)); + } + + @Override + public void grantPermissionOnTopics(List options) throws PulsarAdminException { + sync(() -> grantPermissionOnTopicsAsync(options)); + } + + @Override + public CompletableFuture revokePermissionOnTopicsAsync(List options) { + final WebTarget base = adminV2Namespaces; + WebTarget path = base.path("/revokePermissionsOnTopics"); + return asyncPostRequest(path, Entity.entity(options, MediaType.APPLICATION_JSON)); + } + + @Override + public void revokePermissionOnTopics(List options) throws PulsarAdminException { + sync(() -> revokePermissionOnTopicsAsync(options)); + } + @Override public void revokePermissionsOnNamespace(String namespace, String role) throws PulsarAdminException { sync(() -> revokePermissionsOnNamespaceAsync(namespace, role)); From 667904c6a6631381dc5f4e716278c876a3f3fb83 Mon Sep 17 00:00:00 2001 From: maheshnikam <55378196+nikam14@users.noreply.github.com> Date: Thu, 10 Oct 2024 19:32:42 +0530 Subject: [PATCH 065/327] [Enhancement]Added a metric for geo replication for tracking replicated subscriptions snapshot timeouts (#22381) Co-authored-by: Lari Hotari --- .../persistent/ReplicatedSubscriptionsController.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java index f56cf9de66b75..4fb0022194a02 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java @@ -20,6 +20,7 @@ import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; import io.netty.buffer.ByteBuf; +import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import java.io.IOException; import java.time.Clock; @@ -80,6 +81,14 @@ public class ReplicatedSubscriptionsController implements AutoCloseable, Topic.P "Counter of currently pending snapshots") .register(); + // timeouts use SnapshotOperationResult.TIMEOUT.attributes on the same metric + @PulsarDeprecatedMetric( + newMetricName = OpenTelemetryReplicatedSubscriptionStats.SNAPSHOT_OPERATION_COUNT_METRIC_NAME) + @Deprecated + private static final Counter timedoutSnapshotsMetric = Counter + .build().name("pulsar_replicated_subscriptions_timedout_snapshots") + .help("Counter of timed out snapshots").register(); + private final OpenTelemetryReplicatedSubscriptionStats stats; public ReplicatedSubscriptionsController(PersistentTopic topic, String localCluster) { @@ -263,6 +272,7 @@ private void cleanupTimedOutSnapshots() { } pendingSnapshotsMetric.dec(); + timedoutSnapshotsMetric.inc(); var latencyMillis = entry.getValue().getDurationMillis(); stats.recordSnapshotTimedOut(latencyMillis); it.remove(); From 8b6b3370f921435a61132e4f26f6428907dc69d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 10 Oct 2024 22:03:51 +0800 Subject: [PATCH 066/327] [fix][broker] Avoid orphan ledgers in BucketDelayedDeliveryTracker (#22802) --- .../BucketDelayedDeliveryTrackerFactory.java | 7 ++-- .../BookkeeperBucketSnapshotStorage.java | 18 ++++++++--- .../bucket/BucketDelayedDeliveryTracker.java | 30 +++++++++++++++-- .../bucket/BucketNotExistException.java | 32 +++++++++++++++++++ .../delayed/bucket/ImmutableBucket.java | 10 +++--- 5 files changed, 83 insertions(+), 14 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketNotExistException.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java index c2d002ad19cb0..93eb3ebbc77d5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java @@ -119,10 +119,9 @@ public CompletableFuture cleanResidualSnapshots(ManagedCursor cursor) { FutureUtil.Sequencer sequencer = FutureUtil.Sequencer.create(); cursorProperties.forEach((k, v) -> { if (k != null && v != null && k.startsWith(BucketDelayedDeliveryTracker.DELAYED_BUCKET_KEY_PREFIX)) { - CompletableFuture future = sequencer.sequential(() -> { - return cursor.removeCursorProperty(k) - .thenCompose(__ -> bucketSnapshotStorage.deleteBucketSnapshot(Long.parseLong(v))); - }); + CompletableFuture future = sequencer.sequential(() -> + bucketSnapshotStorage.deleteBucketSnapshot(Long.parseLong(v)) + .thenCompose(__ -> cursor.removeCursorProperty(k))); futures.add(future); } }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java index 8dcfe8d39a8b4..0d90e5e1d980f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java @@ -205,7 +205,10 @@ private CompletableFuture openLedger(Long ledgerId) { BookKeeper.DigestType.fromApiDigestType(config.getManagedLedgerDigestType()), LedgerPassword, (rc, handle, ctx) -> { - if (rc != BKException.Code.OK) { + if (rc == BKException.Code.NoSuchLedgerExistsException) { + // If the ledger does not exist, throw BucketNotExistException + future.completeExceptionally(noSuchLedgerException("Open ledger", ledgerId)); + } else if (rc != BKException.Code.OK) { future.completeExceptionally(bkException("Open ledger", rc, ledgerId)); } else { future.complete(handle); @@ -265,10 +268,11 @@ CompletableFuture> getLedgerEntry(LedgerHandle ledger, private CompletableFuture deleteLedger(long ledgerId) { CompletableFuture future = new CompletableFuture<>(); bookKeeper.asyncDeleteLedger(ledgerId, (int rc, Object cnx) -> { - if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Delete ledger", rc, ledgerId)); - } else { + if (rc == BKException.Code.NoSuchLedgerExistsException || rc == BKException.Code.OK) { + // If the ledger does not exist or has been deleted, we can treat it as success future.complete(null); + } else { + future.completeExceptionally(bkException("Delete ledger", rc, ledgerId)); } }, null); return future; @@ -279,4 +283,10 @@ private static BucketSnapshotPersistenceException bkException(String operation, + " - ledger=" + ledgerId + " - operation=" + operation; return new BucketSnapshotPersistenceException(message); } + + private static BucketNotExistException noSuchLedgerException(String operation, long ledgerId) { + String message = BKException.getMessage(BKException.Code.NoSuchLedgerExistsException) + + " - ledger=" + ledgerId + " - operation=" + operation; + return new BucketNotExistException(message); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index 0091bf5b9bd30..08f3ae1fa6e8a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -180,8 +180,7 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT futures = new HashMap<>(immutableBucketMap.size()); for (Map.Entry, ImmutableBucket> entry : immutableBucketMap.entrySet()) { Range key = entry.getKey(); - ImmutableBucket immutableBucket = entry.getValue(); - futures.put(key, immutableBucket.asyncRecoverBucketSnapshotEntry(this::getCutoffTime)); + futures.put(key, handleRecoverBucketSnapshotEntry(entry.getValue())); } try { @@ -232,6 +231,33 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT return numberDelayedMessages.getValue(); } + /** + * Handle the BucketNotExistException when recover bucket snapshot entry. + * The non exist bucket will be added to `toBeDeletedBucketMap` and deleted from `immutableBuckets` + * in the next step. + * + * @param bucket + * @return + */ + private CompletableFuture> handleRecoverBucketSnapshotEntry(ImmutableBucket bucket) { + CompletableFuture> f = new CompletableFuture<>(); + bucket.asyncRecoverBucketSnapshotEntry(this::getCutoffTime) + .whenComplete((v, e) -> { + if (e == null) { + f.complete(v); + } else { + if (e instanceof BucketNotExistException) { + // If the bucket does not exist, return an empty list, + // the bucket will be deleted from `immutableBuckets` in the next step. + f.complete(Collections.emptyList()); + } else { + f.completeExceptionally(e); + } + } + }); + return f; + } + private synchronized void putAndCleanOverlapRange(Range range, ImmutableBucket immutableBucket, Map, ImmutableBucket> toBeDeletedBucketMap) { RangeMap subRangeMap = immutableBuckets.subRangeMap(range); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketNotExistException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketNotExistException.java new file mode 100644 index 0000000000000..f6c16a1595f54 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketNotExistException.java @@ -0,0 +1,32 @@ +/* + * 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.delayed.bucket; + +import org.apache.pulsar.broker.service.BrokerServiceException; + +public class BucketNotExistException extends BrokerServiceException.PersistenceException { + + public BucketNotExistException(Throwable t) { + super(t); + } + + public BucketNotExistException(String msg) { + super(msg); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/ImmutableBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/ImmutableBucket.java index 0932f51f350ce..a1944a21ea794 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/ImmutableBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/ImmutableBucket.java @@ -193,9 +193,10 @@ CompletableFuture asyncDeleteBucketSnapshot(BucketDelayedMessageIndexStats stats.recordTriggerEvent(BucketDelayedMessageIndexStats.Type.delete); String bucketKey = bucketKey(); long bucketId = getAndUpdateBucketId(); - return removeBucketCursorProperty(bucketKey).thenCompose(__ -> - executeWithRetry(() -> bucketSnapshotStorage.deleteBucketSnapshot(bucketId), - BucketSnapshotPersistenceException.class, MaxRetryTimes)).whenComplete((__, ex) -> { + + return executeWithRetry(() -> bucketSnapshotStorage.deleteBucketSnapshot(bucketId), + BucketSnapshotPersistenceException.class, MaxRetryTimes) + .whenComplete((__, ex) -> { if (ex != null) { log.error("[{}] Failed to delete bucket snapshot, bucketId: {}, bucketKey: {}", dispatcherName, bucketId, bucketKey, ex); @@ -208,7 +209,8 @@ CompletableFuture asyncDeleteBucketSnapshot(BucketDelayedMessageIndexStats stats.recordSuccessEvent(BucketDelayedMessageIndexStats.Type.delete, System.currentTimeMillis() - deleteStartTime); } - }); + }) + .thenCompose(__ -> removeBucketCursorProperty(bucketKey)); } CompletableFuture clear(BucketDelayedMessageIndexStats stats) { From 5506f50fa036f1ad19c3e0abc03e74ecd5c0a665 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 11 Oct 2024 00:14:27 +0800 Subject: [PATCH 067/327] [fix][broker] Fix namespace unload might be blocked too long with extensible load manager (#23433) --- .../pulsar/broker/admin/impl/BrokersBase.java | 6 +++- .../channel/ServiceUnitStateChannelImpl.java | 34 ++++++++++++++++--- .../ExtensibleLoadManagerCloseTest.java | 10 ++++-- 3 files changed, 41 insertions(+), 9 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index da4cee7b4651c..e397dbb64a075 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -388,7 +388,11 @@ public void healthCheck(@Suspended AsyncResponse asyncResponse, asyncResponse.resume(Response.ok("ok").build()); }).exceptionally(ex -> { if (!isRedirectException(ex)) { - LOG.error("[{}] Fail to run health check.", clientAppId(), ex); + if (isNotFoundException(ex)) { + LOG.warn("[{}] Failed to run health check: {}", clientAppId(), ex.getMessage()); + } else { + LOG.error("[{}] Failed to run health check.", clientAppId(), ex); + } } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 49d038d512e59..ea1bf01be5b54 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -87,6 +87,7 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceBundleFactory; @@ -1291,6 +1292,14 @@ private void handleBrokerCreationEvent(String broker) { broker, cleanupJobs.size()); } } + }) + .exceptionally(e -> { + if (FutureUtil.unwrapCompletionException(e) instanceof PulsarAdminException.NotFoundException) { + log.warn("{} Failed to run health check: {}", broker, e.getMessage()); + } else { + log.error("{} Failed to run health check", broker, e); + } + return null; }); } } @@ -1323,12 +1332,19 @@ private void handleBrokerDeletionEvent(String broker) { } } + private boolean channelDisabled() { + final var channelState = this.channelState; + if (channelState == Disabled || channelState == Closed) { + log.warn("[{}] Skip scheduleCleanup because the state is {} now", brokerId, channelState); + return true; + } + return false; + } + private void scheduleCleanup(String broker, long delayInSecs) { var scheduled = new MutableObject>(); try { - final var channelState = this.channelState; - if (channelState == Disabled || channelState == Closed) { - log.warn("[{}] Skip scheduleCleanup because the state is {} now", brokerId, channelState); + if (channelDisabled()) { return; } cleanupJobs.computeIfAbsent(broker, k -> { @@ -1462,6 +1478,10 @@ private CompletableFuture healthCheckBrokerAsync(String brokerId) { } private void doHealthCheckBrokerAsyncWithRetries(String brokerId, int retry, CompletableFuture future) { + if (channelDisabled()) { + future.complete(null); + return; + } try { var admin = getPulsarAdmin(); admin.brokers().healthcheckAsync(TopicVersion.V2, Optional.of(brokerId)) @@ -1472,7 +1492,6 @@ private void doHealthCheckBrokerAsyncWithRetries(String brokerId, int retry, Com return; } if (retry == MAX_BROKER_HEALTH_CHECK_RETRY) { - log.error("Failed health-check broker :{}", brokerId, e); future.completeExceptionally(FutureUtil.unwrapCompletionException(e)); } else { pulsar.getExecutor() @@ -1509,7 +1528,12 @@ private synchronized void doCleanup(String broker, boolean gracefully) { return; } catch (Exception e) { if (debug()) { - log.info("Failed to check broker:{} health", broker, e); + if (e instanceof ExecutionException + && e.getCause() instanceof PulsarAdminException.NotFoundException) { + log.info("The broker {} is not healthy because it's not found", broker); + } else { + log.info("Failed to check broker:{} health", broker, e); + } } log.info("Checked the broker:{} health. Continue the orphan bundle cleanup", broker); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java index ca44f6bc4d6d9..c8427d1a66d53 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java @@ -38,7 +38,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "flaky") +@Test(groups = "broker") public class ExtensibleLoadManagerCloseTest { private static final String clusterName = "test"; @@ -88,14 +88,18 @@ private ServiceConfiguration brokerConfig() { config.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); config.setLoadBalancerDebugModeEnabled(true); config.setBrokerShutdownTimeoutMs(100); + + // Reduce these timeout configs to avoid failed tests being blocked too long + config.setMetadataStoreOperationTimeoutSeconds(5); + config.setNamespaceBundleUnloadingTimeoutMs(5000); return config; } - @Test + @Test(invocationCount = 10) public void testCloseAfterLoadingBundles() throws Exception { setupBrokers(3); - final var topic = "test"; + final var topic = "test-" + System.currentTimeMillis(); final var admin = brokers.get(0).getAdminClient(); admin.topics().createPartitionedTopic(topic, 20); admin.lookups().lookupPartitionedTopic(topic); From bc3e7f60574dd25b0d3511c56d09d233d0c15f42 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 10 Oct 2024 12:30:16 -0700 Subject: [PATCH 068/327] [fix][broker] Allow broker to handle non-recoverable schema error only if SchemaLedgerForceRecovery flag is enabled (#23428) --- .../java/org/apache/pulsar/broker/service/AbstractTopic.java | 3 ++- .../pulsar/broker/service/schema/ClientGetSchemaTest.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) 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 76dd277159cf4..11f00fb28e34b 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 @@ -671,7 +671,8 @@ public CompletableFuture hasSchema() { return brokerService.pulsar().getSchemaRegistryService().getSchema(getSchemaId()).thenApply(Objects::nonNull) .exceptionally(e -> { Throwable ex = e.getCause(); - if (ex instanceof SchemaException || !((SchemaException) ex).isRecoverable()) { + if (brokerService.pulsar().getConfig().isSchemaLedgerForceRecovery() + && (ex instanceof SchemaException && !((SchemaException) ex).isRecoverable())) { return false; } throw ex instanceof CompletionException ? (CompletionException) ex : new CompletionException(ex); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java index ec81f39fef92c..f9c1042b0e97e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java @@ -186,7 +186,7 @@ public void testAddProducerOnDeletedSchemaLedgerTopic() throws Exception { final String topicOne = "test-deleted-schema-ledger"; final String fqtnOne = TopicName.get(TopicDomain.persistent.value(), tenant, namespace, topicOne).toString(); - //pulsar.getConfig().setManagedLedgerForceRecovery(true); + pulsar.getConfig().setSchemaLedgerForceRecovery(true); admin.namespaces().createNamespace(tenant + "/" + namespace, Sets.newHashSet("test")); // (1) create topic with schema From 2dace760b03bd679dae5ea682e1e23fd93c15f78 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Fri, 11 Oct 2024 09:16:50 +0800 Subject: [PATCH 069/327] [improve][client] Add log when can't add message to the container (#21924) --- .../java/org/apache/pulsar/client/impl/MessagesImpl.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java index d4cd36a22e15f..9768fd7c74b0f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java @@ -22,11 +22,13 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import lombok.extern.slf4j.Slf4j; import net.jcip.annotations.NotThreadSafe; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Messages; @NotThreadSafe +@Slf4j public class MessagesImpl implements Messages { private final List> messageList; @@ -49,10 +51,14 @@ protected boolean canAdd(Message message) { return true; } if (maxNumberOfMessages > 0 && currentNumberOfMessages + 1 > maxNumberOfMessages) { + log.warn("can't add message to the container, has exceeded the maxNumberOfMessages : {} ", + maxNumberOfMessages); return false; } if (maxSizeOfMessages > 0 && currentSizeOfMessages + message.size() > maxSizeOfMessages) { + log.warn("can't add message to the container, has exceeded the maxSizeOfMessages : {} ", + maxSizeOfMessages); return false; } From 50dc521f7f97ec0151036a51978db74f5cd8ca68 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 11 Oct 2024 23:42:39 +0800 Subject: [PATCH 070/327] [fix][broker] normalize path (#23438) Signed-off-by: Zixuan Liu --- .../org/apache/bookkeeper/mledger/offload/OffloaderUtils.java | 2 +- .../broker/web/plugin/servlet/AdditionalServletUtils.java | 4 ++-- .../pulsar/broker/intercept/BrokerInterceptorUtils.java | 4 ++-- .../apache/pulsar/broker/protocol/ProtocolHandlerUtils.java | 4 ++-- .../pulsar/broker/service/plugin/EntryFilterProvider.java | 4 ++-- .../pulsar/admin/cli/utils/CustomCommandFactoryProvider.java | 4 ++-- .../main/java/org/apache/pulsar/functions/LocalRunner.java | 2 +- .../pulsar/functions/utils/functions/FunctionUtils.java | 2 +- .../org/apache/pulsar/functions/utils/io/ConnectorUtils.java | 2 +- .../pulsar/functions/worker/service/WorkerServiceLoader.java | 2 +- .../apache/pulsar/proxy/extensions/ProxyExtensionsUtils.java | 4 ++-- 11 files changed, 17 insertions(+), 17 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloaderUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloaderUtils.java index 7af3680d880b2..d160189188a18 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloaderUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloaderUtils.java @@ -125,7 +125,7 @@ public static OffloaderDefinition getOffloaderDefinition(String narPath, String public static Offloaders searchForOffloaders(String offloadersPath, String narExtractionDirectory) throws IOException { - Path path = Paths.get(offloadersPath).toAbsolutePath(); + Path path = Paths.get(offloadersPath).toAbsolutePath().normalize(); log.info("Searching for offloaders in {}", path); Offloaders offloaders = new Offloaders(); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletUtils.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletUtils.java index bfbca2889e289..c3d18833de5a8 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletUtils.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletUtils.java @@ -74,7 +74,7 @@ private AdditionalServletDefinition getAdditionalServletDefinition(NarClassLoade */ public AdditionalServletDefinitions searchForServlets(String additionalServletDirectory, String narExtractionDirectory) throws IOException { - Path path = Paths.get(additionalServletDirectory).toAbsolutePath(); + Path path = Paths.get(additionalServletDirectory).toAbsolutePath().normalize(); log.info("Searching for additional servlets in {}", path); AdditionalServletDefinitions servletDefinitions = new AdditionalServletDefinitions(); @@ -119,7 +119,7 @@ public AdditionalServletDefinitions searchForServlets(String additionalServletDi public AdditionalServletWithClassLoader load( AdditionalServletMetadata metadata, String narExtractionDirectory) throws IOException { - final File narFile = metadata.getArchivePath().toAbsolutePath().toFile(); + final File narFile = metadata.getArchivePath().toAbsolutePath().normalize().toFile(); NarClassLoader ncl = NarClassLoaderBuilder.builder() .narFile(narFile) .parentClassLoader(AdditionalServlet.class.getClassLoader()) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtils.java index 990d7048e6ee1..0d0d33fa1dfc6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtils.java @@ -75,7 +75,7 @@ private BrokerInterceptorDefinition getBrokerInterceptorDefinition(NarClassLoade */ public BrokerInterceptorDefinitions searchForInterceptors(String interceptorsDirectory, String narExtractionDirectory) throws IOException { - Path path = Paths.get(interceptorsDirectory).toAbsolutePath(); + Path path = Paths.get(interceptorsDirectory).toAbsolutePath().normalize(); log.info("Searching for broker interceptors in {}", path); BrokerInterceptorDefinitions interceptors = new BrokerInterceptorDefinitions(); @@ -119,7 +119,7 @@ public BrokerInterceptorDefinitions searchForInterceptors(String interceptorsDir */ BrokerInterceptorWithClassLoader load(BrokerInterceptorMetadata metadata, String narExtractionDirectory) throws IOException { - final File narFile = metadata.getArchivePath().toAbsolutePath().toFile(); + final File narFile = metadata.getArchivePath().toAbsolutePath().normalize().toFile(); NarClassLoader ncl = NarClassLoaderBuilder.builder() .narFile(narFile) .parentClassLoader(BrokerInterceptorUtils.class.getClassLoader()) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerUtils.java index ff33326169a4d..b66f493555886 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerUtils.java @@ -75,7 +75,7 @@ private static ProtocolHandlerDefinition getProtocolHandlerDefinition(NarClassLo */ public static ProtocolHandlerDefinitions searchForHandlers(String handlersDirectory, String narExtractionDirectory) throws IOException { - Path path = Paths.get(handlersDirectory).toAbsolutePath(); + Path path = Paths.get(handlersDirectory).toAbsolutePath().normalize(); log.info("Searching for protocol handlers in {}", path); ProtocolHandlerDefinitions handlers = new ProtocolHandlerDefinitions(); @@ -119,7 +119,7 @@ public static ProtocolHandlerDefinitions searchForHandlers(String handlersDirect */ static ProtocolHandlerWithClassLoader load(ProtocolHandlerMetadata metadata, String narExtractionDirectory) throws IOException { - final File narFile = metadata.getArchivePath().toAbsolutePath().toFile(); + final File narFile = metadata.getArchivePath().toAbsolutePath().normalize().toFile(); NarClassLoader ncl = NarClassLoaderBuilder.builder() .narFile(narFile) .parentClassLoader(ProtocolHandler.class.getClassLoader()) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java index 53418744b5486..297c124c60525 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java @@ -122,7 +122,7 @@ public List getBrokerEntryFilters() { private void initialize() throws IOException { final String entryFiltersDirectory = serviceConfiguration.getEntryFiltersDirectory(); - Path path = Paths.get(entryFiltersDirectory).toAbsolutePath(); + Path path = Paths.get(entryFiltersDirectory).toAbsolutePath().normalize(); log.info("Searching for entry filters in {}", path); @@ -217,7 +217,7 @@ private NarClassLoader loadNarClassLoader(Path archivePath) { return cachedClassLoaders .computeIfAbsent(absolutePath, narFilePath -> { try { - final File narFile = archivePath.toAbsolutePath().toFile(); + final File narFile = archivePath.toAbsolutePath().normalize().toFile(); return NarClassLoaderBuilder.builder() .narFile(narFile) .parentClassLoader(EntryFilter.class.getClassLoader()) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CustomCommandFactoryProvider.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CustomCommandFactoryProvider.java index 23b7e9eb58d42..5c6566ac8ce1e 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CustomCommandFactoryProvider.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CustomCommandFactoryProvider.java @@ -77,7 +77,7 @@ public static List createCustomCommandFactories( private static CustomCommandFactoryDefinitions searchForCustomCommandFactories(String directory, String narExtractionDirectory) throws IOException { - Path path = Paths.get(directory).toAbsolutePath(); + Path path = Paths.get(directory).toAbsolutePath().normalize(); log.debug("Searching for command factories in {}", path); CustomCommandFactoryDefinitions customCommandFactoryDefinitions = new CustomCommandFactoryDefinitions(); @@ -142,7 +142,7 @@ static CustomCommandFactoryDefinition getCustomCommandFactoryDefinition(NarClass private static CustomCommandFactory load(CustomCommandFactoryMetaData metadata, String narExtractionDirectory) throws IOException { - final File narFile = metadata.getArchivePath().toAbsolutePath().toFile(); + final File narFile = metadata.getArchivePath().toAbsolutePath().normalize().toFile(); NarClassLoader ncl = NarClassLoaderBuilder.builder() .narFile(narFile) .parentClassLoader(CustomCommandFactory.class.getClassLoader()) diff --git a/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java b/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java index 3b1c86a68c285..1e2d4e0b21b32 100644 --- a/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java +++ b/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java @@ -278,7 +278,7 @@ private static String getPulsarDirectory(String directory) { } else { directoryPath = Path.of(directory); } - return directoryPath.toAbsolutePath().toString(); + return directoryPath.toAbsolutePath().normalize().toString(); } private static File createNarExtractionTempDirectory() { diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionUtils.java index 31a5540e0bfaf..24f801f3da26f 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionUtils.java @@ -78,7 +78,7 @@ public static T getPulsarIOServiceConfig(NarClassLoader narClassLoader, Clas public static TreeMap searchForFunctions(String functionsDirectory, String narExtractionDirectory, boolean enableClassloading) throws IOException { - Path path = Paths.get(functionsDirectory).toAbsolutePath(); + Path path = Paths.get(functionsDirectory).toAbsolutePath().normalize(); log.info("Searching for functions in {}", path); TreeMap functions = new TreeMap<>(); diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/ConnectorUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/ConnectorUtils.java index df1310965f392..71cab749ba06c 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/ConnectorUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/ConnectorUtils.java @@ -156,7 +156,7 @@ private static List getAllFields(TypeDefinition type) { public static TreeMap searchForConnectors(String connectorsDirectory, String narExtractionDirectory, boolean enableClassloading) throws IOException { - Path path = Paths.get(connectorsDirectory).toAbsolutePath(); + Path path = Paths.get(connectorsDirectory).toAbsolutePath().normalize(); log.info("Searching for connectors in {}", path); TreeMap connectors = new TreeMap<>(); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceLoader.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceLoader.java index 8c6535342c3d3..76e7081fe637c 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceLoader.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceLoader.java @@ -73,7 +73,7 @@ private static WorkerServiceDefinition getWorkerServiceDefinition(NarClassLoader */ static WorkerServiceWithClassLoader load(WorkerServiceMetadata metadata, String narExtractionDirectory) throws IOException { - final File narFile = metadata.getArchivePath().toAbsolutePath().toFile(); + final File narFile = metadata.getArchivePath().toAbsolutePath().normalize().toFile(); NarClassLoader ncl = NarClassLoaderBuilder.builder() .narFile(narFile) .parentClassLoader(WorkerService.class.getClassLoader()) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/extensions/ProxyExtensionsUtils.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/extensions/ProxyExtensionsUtils.java index b488fea9dac5d..d2b9f91b37bb5 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/extensions/ProxyExtensionsUtils.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/extensions/ProxyExtensionsUtils.java @@ -75,7 +75,7 @@ private static ProxyExtensionDefinition getProxyExtensionDefinition(NarClassLoad */ public static ExtensionsDefinitions searchForExtensions(String extensionsDirectory, String narExtractionDirectory) throws IOException { - Path path = Paths.get(extensionsDirectory).toAbsolutePath(); + Path path = Paths.get(extensionsDirectory).toAbsolutePath().normalize(); log.info("Searching for extensions in {}", path); ExtensionsDefinitions extensions = new ExtensionsDefinitions(); @@ -119,7 +119,7 @@ public static ExtensionsDefinitions searchForExtensions(String extensionsDirecto */ static ProxyExtensionWithClassLoader load(ProxyExtensionMetadata metadata, String narExtractionDirectory) throws IOException { - final File narFile = metadata.getArchivePath().toAbsolutePath().toFile(); + final File narFile = metadata.getArchivePath().toAbsolutePath().normalize().toFile(); NarClassLoader ncl = NarClassLoaderBuilder.builder() .narFile(narFile) .parentClassLoader(ProxyExtension.class.getClassLoader()) From d5e3675439458faa6f1d75929719d5a80e34238a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 11 Oct 2024 19:22:35 +0300 Subject: [PATCH 071/327] [fix][sec] Drop hdfs2 support, Upgrade hadoop3 to 3.4.0 and dnsjava to 3.6.2 to address CVE-2024-25638 (#23411) --- .github/workflows/pulsar-ci.yaml | 2 +- .../terraform-ansible/deploy-pulsar.yaml | 1 - distribution/io/src/assemble/io.xml | 1 - pom.xml | 60 ++++- pulsar-bom/pom.xml | 5 - pulsar-io/docs/pom.xml | 5 - pulsar-io/hdfs2/pom.xml | 130 --------- .../pulsar/io/hdfs2/AbstractHdfsConfig.java | 76 ------ .../io/hdfs2/AbstractHdfsConnector.java | 246 ------------------ .../apache/pulsar/io/hdfs2/Compression.java | 26 -- .../apache/pulsar/io/hdfs2/HdfsResources.java | 51 ---- .../apache/pulsar/io/hdfs2/SecurityUtil.java | 90 ------- .../apache/pulsar/io/hdfs2/package-info.java | 19 -- .../io/hdfs2/sink/HdfsAbstractSink.java | 124 --------- .../pulsar/io/hdfs2/sink/HdfsSinkConfig.java | 117 --------- .../pulsar/io/hdfs2/sink/HdfsSyncThread.java | 79 ------ .../pulsar/io/hdfs2/sink/package-info.java | 19 -- .../seq/HdfsAbstractSequenceFileSink.java | 95 ------- .../sink/seq/HdfsSequentialTextSink.java | 70 ----- .../io/hdfs2/sink/seq/HdfsTextSink.java | 53 ---- .../io/hdfs2/sink/seq/package-info.java | 19 -- .../sink/text/HdfsAbstractTextFileSink.java | 78 ------ .../io/hdfs2/sink/text/HdfsStringSink.java | 34 --- .../io/hdfs2/sink/text/package-info.java | 19 -- .../META-INF/services/pulsar-io.yaml | 23 -- .../src/main/resources/findbugsExclude.xml | 58 ----- .../io/hdfs2/sink/AbstractHdfsSinkTest.java | 120 --------- .../io/hdfs2/sink/HdfsSinkConfigTests.java | 158 ----------- .../sink/seq/HdfsSequentialSinkTests.java | 110 -------- .../io/hdfs2/sink/seq/HdfsTextSinkTests.java | 122 --------- .../hdfs2/sink/text/HdfsStringSinkTests.java | 118 --------- .../src/test/resources/hadoop/core-site.xml | 32 --- .../src/test/resources/hadoop/hdfs-site.xml | 34 --- .../hdfs2/src/test/resources/sinkConfig.yaml | 26 -- pulsar-io/hdfs3/pom.xml | 50 ++-- pulsar-io/pom.xml | 2 - tiered-storage/file-system/pom.xml | 12 + 37 files changed, 94 insertions(+), 2190 deletions(-) delete mode 100644 pulsar-io/hdfs2/pom.xml delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/AbstractHdfsConfig.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/AbstractHdfsConnector.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/Compression.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/HdfsResources.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/SecurityUtil.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/package-info.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsAbstractSink.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsSinkConfig.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsSyncThread.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/package-info.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/seq/HdfsAbstractSequenceFileSink.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/seq/HdfsSequentialTextSink.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/seq/HdfsTextSink.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/seq/package-info.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/text/HdfsAbstractTextFileSink.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/text/HdfsStringSink.java delete mode 100644 pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/text/package-info.java delete mode 100644 pulsar-io/hdfs2/src/main/resources/META-INF/services/pulsar-io.yaml delete mode 100644 pulsar-io/hdfs2/src/main/resources/findbugsExclude.xml delete mode 100644 pulsar-io/hdfs2/src/test/java/org/apache/pulsar/io/hdfs2/sink/AbstractHdfsSinkTest.java delete mode 100644 pulsar-io/hdfs2/src/test/java/org/apache/pulsar/io/hdfs2/sink/HdfsSinkConfigTests.java delete mode 100644 pulsar-io/hdfs2/src/test/java/org/apache/pulsar/io/hdfs2/sink/seq/HdfsSequentialSinkTests.java delete mode 100644 pulsar-io/hdfs2/src/test/java/org/apache/pulsar/io/hdfs2/sink/seq/HdfsTextSinkTests.java delete mode 100644 pulsar-io/hdfs2/src/test/java/org/apache/pulsar/io/hdfs2/sink/text/HdfsStringSinkTests.java delete mode 100644 pulsar-io/hdfs2/src/test/resources/hadoop/core-site.xml delete mode 100644 pulsar-io/hdfs2/src/test/resources/hadoop/hdfs-site.xml delete mode 100644 pulsar-io/hdfs2/src/test/resources/sinkConfig.yaml diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index bf44c51b6ad02..87d8cd7cf9a78 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -1498,7 +1498,7 @@ jobs: - name: trigger dependency check run: | mvn -B -ntp verify -PskipDocker,skip-all,owasp-dependency-check -Dcheckstyle.skip=true -DskipTests \ - -pl '!distribution/server,!distribution/io,!distribution/offloaders,!tiered-storage/file-system,!pulsar-io/flume,!pulsar-io/hbase,!pulsar-io/hdfs2,!pulsar-io/hdfs3,!pulsar-io/docs,!pulsar-io/jdbc/openmldb' + -pl '!distribution/server,!distribution/io,!distribution/offloaders,!tiered-storage/file-system,!pulsar-io/flume,!pulsar-io/hbase,!pulsar-io/hdfs3,!pulsar-io/docs,!pulsar-io/jdbc/openmldb' - name: Upload report uses: actions/upload-artifact@v4 diff --git a/deployment/terraform-ansible/deploy-pulsar.yaml b/deployment/terraform-ansible/deploy-pulsar.yaml index db2fd1257ca41..3a9f0fd942c17 100644 --- a/deployment/terraform-ansible/deploy-pulsar.yaml +++ b/deployment/terraform-ansible/deploy-pulsar.yaml @@ -147,7 +147,6 @@ # - file # - flume # - hbase -# - hdfs2 # - hdfs3 # - influxdb # - jdbc-clickhouse diff --git a/distribution/io/src/assemble/io.xml b/distribution/io/src/assemble/io.xml index f98ee14bb20c9..cf7731b4c85ab 100644 --- a/distribution/io/src/assemble/io.xml +++ b/distribution/io/src/assemble/io.xml @@ -63,7 +63,6 @@ ${basedir}/../../pulsar-io/kafka-connect-adaptor-nar/target/pulsar-io-kafka-connect-adaptor-${project.version}.nar ${basedir}/../../pulsar-io/hbase/target/pulsar-io-hbase-${project.version}.nar ${basedir}/../../pulsar-io/kinesis/target/pulsar-io-kinesis-${project.version}.nar - ${basedir}/../../pulsar-io/hdfs2/target/pulsar-io-hdfs2-${project.version}.nar ${basedir}/../../pulsar-io/hdfs3/target/pulsar-io-hdfs3-${project.version}.nar ${basedir}/../../pulsar-io/file/target/pulsar-io-file-${project.version}.nar ${basedir}/../../pulsar-io/data-generator/target/pulsar-io-data-generator-${project.version}.nar diff --git a/pom.xml b/pom.xml index f99eb3066d5e6..b89dd1597cc84 100644 --- a/pom.xml +++ b/pom.xml @@ -196,7 +196,6 @@ flexible messaging model and an intuitive client API. 0.4.6 2.7.5 0.4.4-hotfix1 - 3.3.5 2.4.10 2.16.0 8.12.1 @@ -207,9 +206,10 @@ flexible messaging model and an intuitive client API. 1.15.16.Final 0.11.1 0.28.0 - 2.10.2 - 3.3.5 - 2.4.16 + 3.4.0 + 3.6.2 + ${hadoop3.version} + 2.6.0-hadoop3 32.1.2-jre 1.0 0.16.1 @@ -1313,6 +1313,58 @@ flexible messaging model and an intuitive client API. ${commons.collections4.version} + + + org.apache.hadoop + hadoop-common + ${hadoop3.version} + + + dnsjava + dnsjava + + + + + org.apache.hadoop + hadoop-auth + ${hadoop3.version} + + + dnsjava + dnsjava + + + + + org.apache.hadoop + hadoop-client + ${hadoop3.version} + + + dnsjava + dnsjava + + + + + org.apache.hbase + hbase-client + ${hbase.version} + + + dnsjava + dnsjava + + + + + + dnsjava + dnsjava + ${dnsjava3.version} + + com.lmax diff --git a/pulsar-bom/pom.xml b/pulsar-bom/pom.xml index d195411fa6479..e674301f18a3a 100644 --- a/pulsar-bom/pom.xml +++ b/pulsar-bom/pom.xml @@ -495,11 +495,6 @@ pulsar-io-hbase ${project.version} - - org.apache.pulsar - pulsar-io-hdfs2 - ${project.version} - org.apache.pulsar pulsar-io-hdfs3 diff --git a/pulsar-io/docs/pom.xml b/pulsar-io/docs/pom.xml index ac4ae9496d1bb..e373db26c450d 100644 --- a/pulsar-io/docs/pom.xml +++ b/pulsar-io/docs/pom.xml @@ -127,11 +127,6 @@ pulsar-io-hbase ${project.version} - - ${project.groupId} - pulsar-io-hdfs2 - ${project.version} - ${project.groupId} pulsar-io-hdfs3 diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml deleted file mode 100644 index d5fb33c170db1..0000000000000 --- a/pulsar-io/hdfs2/pom.xml +++ /dev/null @@ -1,130 +0,0 @@ - - - 4.0.0 - - org.apache.pulsar - pulsar-io - 4.0.0-SNAPSHOT - - pulsar-io-hdfs2 - Pulsar IO :: Hdfs2 - - - - ${project.groupId} - pulsar-io-core - ${project.version} - - - - com.fasterxml.jackson.core - jackson-databind - - - - com.fasterxml.jackson.dataformat - jackson-dataformat-yaml - - - - org.apache.commons - commons-collections4 - - - - org.apache.hadoop - hadoop-client - ${hadoop2.version} - - - log4j - log4j - - - org.slf4j - * - - - org.apache.avro - avro - - - - - org.apache.commons - commons-lang3 - - - - - - - org.apache.nifi - nifi-nar-maven-plugin - - - com.github.spotbugs - spotbugs-maven-plugin - ${spotbugs-maven-plugin.version} - - ${basedir}/src/main/resources/findbugsExclude.xml - - - - spotbugs - verify - - check - - - - - - - - - - owasp-dependency-check - - - - org.owasp - dependency-check-maven - - - - aggregate - - none - - - - - - - - - \ No newline at end of file diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/AbstractHdfsConfig.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/AbstractHdfsConfig.java deleted file mode 100644 index 757360e04533c..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/AbstractHdfsConfig.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2; - -import java.io.Serializable; -import lombok.Data; -import lombok.experimental.Accessors; -import org.apache.commons.lang.StringUtils; - -/** - * Configuration object for all HDFS components. - */ -@Data -@Accessors(chain = true) -public abstract class AbstractHdfsConfig implements Serializable { - - private static final long serialVersionUID = 1L; - - /** - * A file or comma separated list of files which contains the Hadoop file system configuration, - * e.g. 'core-site.xml', 'hdfs-site.xml'. - */ - private String hdfsConfigResources; - - /** - * The HDFS directory from which files should be read from or written to. - */ - private String directory; - - /** - * The character encoding for the files, e.g. UTF-8, ASCII, etc. - */ - private String encoding; - - /** - * The compression codec used to compress/de-compress the files on HDFS. - */ - private Compression compression; - - /** - * The Kerberos user principal account to use for authentication. - */ - private String kerberosUserPrincipal; - - /** - * The full pathname to the Kerberos keytab file to use for authentication. - */ - private String keytab; - - public void validate() { - if (StringUtils.isEmpty(hdfsConfigResources) || StringUtils.isEmpty(directory)) { - throw new IllegalArgumentException("Required property not set."); - } - - if ((StringUtils.isNotEmpty(kerberosUserPrincipal) && StringUtils.isEmpty(keytab)) - || (StringUtils.isEmpty(kerberosUserPrincipal) && StringUtils.isNotEmpty(keytab))) { - throw new IllegalArgumentException("Values for both kerberosUserPrincipal & keytab are required."); - } - } -} diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/AbstractHdfsConnector.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/AbstractHdfsConnector.java deleted file mode 100644 index d7277aa627383..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/AbstractHdfsConnector.java +++ /dev/null @@ -1,246 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2; - -import java.io.IOException; -import java.lang.ref.WeakReference; -import java.net.InetSocketAddress; -import java.net.Socket; -import java.net.URI; -import java.nio.charset.Charset; -import java.security.PrivilegedExceptionAction; -import java.util.Collections; -import java.util.Map; -import java.util.WeakHashMap; -import java.util.concurrent.atomic.AtomicReference; -import javax.net.SocketFactory; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.CompressionCodecFactory; -import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.pulsar.io.hdfs2.sink.HdfsSinkConfig; - -/** - * A Simple abstract class for HDFS connectors. - * Provides methods for connecting to HDFS - */ -public abstract class AbstractHdfsConnector { - - private static final Object RESOURCES_LOCK = new Object(); - - // Hadoop Configuration, Filesystem, and UserGroupInformation (optional) - protected final AtomicReference hdfsResources = new AtomicReference<>(); - protected AbstractHdfsConfig connectorConfig; - protected CompressionCodecFactory compressionCodecFactory; - - public AbstractHdfsConnector() { - hdfsResources.set(new HdfsResources(null, null, null)); - } - - /* - * Reset Hadoop Configuration and FileSystem based on the supplied configuration resources. - */ - protected HdfsResources resetHDFSResources(HdfsSinkConfig hdfsSinkConfig) throws IOException { - Configuration config = new ExtendedConfiguration(); - config.setClassLoader(Thread.currentThread().getContextClassLoader()); - - getConfig(config, connectorConfig.getHdfsConfigResources()); - - // first check for timeout on HDFS connection, because FileSystem has a hard coded 15 minute timeout - checkHdfsUriForTimeout(config); - - /* Disable caching of Configuration and FileSystem objects, else we cannot reconfigure - * the processor without a complete restart - */ - String disableCacheName = String.format("fs.%s.impl.disable.cache", - FileSystem.getDefaultUri(config).getScheme()); - config.set(disableCacheName, "true"); - - // If kerberos is enabled, create the file system as the kerberos principal - // -- use RESOURCE_LOCK to guarantee UserGroupInformation is accessed by only a single thread at at time - FileSystem fs; - UserGroupInformation ugi; - synchronized (RESOURCES_LOCK) { - if (SecurityUtil.isSecurityEnabled(config)) { - ugi = SecurityUtil.loginKerberos(config, - connectorConfig.getKerberosUserPrincipal(), connectorConfig.getKeytab()); - fs = getFileSystemAsUser(config, ugi); - } else { - config.set("ipc.client.fallback-to-simple-auth-allowed", "true"); - config.set("hadoop.security.authentication", "simple"); - ugi = SecurityUtil.loginSimple(config); - fs = getFileSystemAsUser(config, ugi); - } - } - return new HdfsResources(config, fs, ugi); - } - - private static Configuration getConfig(final Configuration config, String res) throws IOException { - boolean foundResources = false; - if (null != res) { - String[] resources = res.split(","); - for (String resource : resources) { - config.addResource(new Path(resource.trim())); - foundResources = true; - } - } - - if (!foundResources) { - // check that at least 1 non-default resource is available on the classpath - String configStr = config.toString(); - for (String resource : configStr.substring(configStr.indexOf(":") + 1).split(",")) { - if (!resource.contains("default") && config.getResource(resource.trim()) != null) { - foundResources = true; - break; - } - } - } - - if (!foundResources) { - throw new IOException("Could not find any of the " + res + " on the classpath"); - } - return config; - } - - /* - * Reduce the timeout of a socket connection from the default in FileSystem.get() - */ - protected void checkHdfsUriForTimeout(Configuration config) throws IOException { - URI hdfsUri = FileSystem.getDefaultUri(config); - String address = hdfsUri.getAuthority(); - int port = hdfsUri.getPort(); - if (address == null || address.isEmpty() || port < 0) { - return; - } - InetSocketAddress namenode = NetUtils.createSocketAddr(address, port); - SocketFactory socketFactory = NetUtils.getDefaultSocketFactory(config); - try (Socket socket = socketFactory.createSocket()) { - NetUtils.connect(socket, namenode, 1000); // 1 second timeout - } - } - - /** - * This exists in order to allow unit tests to override it so that they don't take several - * minutes waiting for UDP packets to be received. - * - * @param config - * the configuration to use - * @return the FileSystem that is created for the given Configuration - * @throws IOException - * if unable to create the FileSystem - */ - protected FileSystem getFileSystem(final Configuration config) throws IOException { - return FileSystem.get(config); - } - - protected FileSystem getFileSystemAsUser(final Configuration config, UserGroupInformation ugi) throws IOException { - try { - return ugi.doAs((PrivilegedExceptionAction) () -> FileSystem.get(config)); - } catch (InterruptedException e) { - throw new IOException("Unable to create file system: " + e.getMessage()); - } - } - - protected Configuration getConfiguration() { - return hdfsResources.get().getConfiguration(); - } - - protected FileSystem getFileSystem() { - return hdfsResources.get().getFileSystem(); - } - - protected UserGroupInformation getUserGroupInformation() { - return hdfsResources.get().getUserGroupInformation(); - } - - protected String getEncoding() { - return StringUtils.isNotBlank(connectorConfig.getEncoding()) - ? connectorConfig.getEncoding() : Charset.defaultCharset().name(); - } - - protected CompressionCodec getCompressionCodec() { - if (connectorConfig.getCompression() == null) { - return null; - } - - CompressionCodec codec = getCompressionCodecFactory() - .getCodecByName(connectorConfig.getCompression().name()); - - return (codec != null) ? codec : new DefaultCodec(); - } - - protected CompressionCodecFactory getCompressionCodecFactory() { - if (compressionCodecFactory == null) { - compressionCodecFactory = new CompressionCodecFactory(getConfiguration()); - } - - return compressionCodecFactory; - } - - /** - * Extending Hadoop Configuration to prevent it from caching classes that can't be found. Since users may be - * adding additional JARs to the classpath we don't want them to have to restart the JVM to be able to load - * something that was previously not found, but might now be available. - * Reference the original getClassByNameOrNull from Configuration. - */ - static class ExtendedConfiguration extends Configuration { - - private final Map>>> cacheClasses = new WeakHashMap<>(); - - @Override - public Class getClassByNameOrNull(String name) { - final ClassLoader classLoader = getClassLoader(); - - Map>> map; - synchronized (cacheClasses) { - map = cacheClasses.get(classLoader); - if (map == null) { - map = Collections.synchronizedMap(new WeakHashMap<>()); - cacheClasses.put(classLoader, map); - } - } - - Class clazz = null; - WeakReference> ref = map.get(name); - if (ref != null) { - clazz = ref.get(); - } - - if (clazz == null) { - try { - clazz = Class.forName(name, true, classLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - return null; - } - // two putters can race here, but they'll put the same class - map.put(name, new WeakReference<>(clazz)); - return clazz; - } else { - // cache hit - return clazz; - } - } - - } -} diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/Compression.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/Compression.java deleted file mode 100644 index 1e3d2f9490439..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/Compression.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2; - -/** - * An enumeration of compression codecs available for HDFS. - */ -public enum Compression { - BZIP2, DEFLATE, GZIP, LZ4, SNAPPY, ZSTANDARD -} diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/HdfsResources.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/HdfsResources.java deleted file mode 100644 index 5fd6b283e6b41..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/HdfsResources.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.security.UserGroupInformation; - -/** - * A wrapper class for HDFS resources. - */ -public class HdfsResources { - - private final Configuration configuration; - private final FileSystem fileSystem; - private final UserGroupInformation userGroupInformation; - - public HdfsResources(Configuration config, FileSystem fs, UserGroupInformation ugi) { - this.configuration = config; - this.fileSystem = fs; - this.userGroupInformation = ugi; - } - - public Configuration getConfiguration() { - return configuration; - } - - public FileSystem getFileSystem() { - return fileSystem; - } - - public UserGroupInformation getUserGroupInformation() { - return userGroupInformation; - } -} diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/SecurityUtil.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/SecurityUtil.java deleted file mode 100644 index ca178aad911e2..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/SecurityUtil.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2; - -import java.io.IOException; -import org.apache.commons.lang3.Validate; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.UserGroupInformation; - -/** - * Provides synchronized access to UserGroupInformation to avoid multiple processors/services from - * interfering with each other. - */ -public class SecurityUtil { - public static final String HADOOP_SECURITY_AUTHENTICATION = "hadoop.security.authentication"; - public static final String KERBEROS = "kerberos"; - - /** - * Initializes UserGroupInformation with the given Configuration and performs the login for the - * given principal and keytab. All logins should happen through this class to ensure other threads - * are not concurrently modifying UserGroupInformation. - *

- * @param config the configuration instance - * @param principal the principal to authenticate as - * @param keyTab the keytab to authenticate with - * - * @return the UGI for the given principal - * - * @throws IOException if login failed - */ - public static synchronized UserGroupInformation loginKerberos(final Configuration config, - final String principal, final String keyTab) throws IOException { - Validate.notNull(config); - Validate.notNull(principal); - Validate.notNull(keyTab); - - UserGroupInformation.setConfiguration(config); - UserGroupInformation.loginUserFromKeytab(principal.trim(), keyTab.trim()); - return UserGroupInformation.getCurrentUser(); - } - - /** - * Initializes UserGroupInformation with the given Configuration and - * returns UserGroupInformation.getLoginUser(). All logins should happen - * through this class to ensure other threads are not concurrently - * modifying UserGroupInformation. - * - * @param config the configuration instance - * - * @return the UGI for the given principal - * - * @throws IOException if login failed - */ - public static synchronized UserGroupInformation loginSimple(final Configuration config) throws IOException { - Validate.notNull(config); - UserGroupInformation.setConfiguration(config); - return UserGroupInformation.getLoginUser(); - } - - /** - * Initializes UserGroupInformation with the given Configuration and returns - * UserGroupInformation.isSecurityEnabled(). - * All checks for isSecurityEnabled() should happen through this method. - * - * @param config the given configuration - * - * @return true if kerberos is enabled on the given configuration, false otherwise - * - */ - public static boolean isSecurityEnabled(final Configuration config) { - Validate.notNull(config); - return KERBEROS.equalsIgnoreCase(config.get(HADOOP_SECURITY_AUTHENTICATION)); - } -} diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/package-info.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/package-info.java deleted file mode 100644 index 464c6db341e8f..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2; \ No newline at end of file diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsAbstractSink.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsAbstractSink.java deleted file mode 100644 index 7b025d16378ff..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsAbstractSink.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2.sink; - -import java.io.IOException; -import java.time.LocalDateTime; -import java.time.format.DateTimeFormatter; -import java.util.Map; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.io.FilenameUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.pulsar.functions.api.Record; -import org.apache.pulsar.io.core.KeyValue; -import org.apache.pulsar.io.core.Sink; -import org.apache.pulsar.io.core.SinkContext; -import org.apache.pulsar.io.hdfs2.AbstractHdfsConnector; -import org.apache.pulsar.io.hdfs2.HdfsResources; - -/** - * A Simple abstract class for HDFS sink. - * Users need to implement extractKeyValue function to use this sink. - */ -@Slf4j -public abstract class HdfsAbstractSink extends AbstractHdfsConnector implements Sink { - - protected HdfsSinkConfig hdfsSinkConfig; - protected BlockingQueue> unackedRecords; - protected HdfsSyncThread syncThread; - private Path path; - private FSDataOutputStream hdfsStream; - private DateTimeFormatter subdirectoryFormatter; - - public abstract KeyValue extractKeyValue(Record record); - protected abstract void createWriter() throws IOException; - - @Override - public void open(Map config, SinkContext sinkContext) throws Exception { - hdfsSinkConfig = HdfsSinkConfig.load(config); - hdfsSinkConfig.validate(); - connectorConfig = hdfsSinkConfig; - unackedRecords = new LinkedBlockingQueue> (hdfsSinkConfig.getMaxPendingRecords()); - if (hdfsSinkConfig.getSubdirectoryPattern() != null) { - subdirectoryFormatter = DateTimeFormatter.ofPattern(hdfsSinkConfig.getSubdirectoryPattern()); - } - connectToHdfs(); - createWriter(); - launchSyncThread(); - } - - @Override - public void close() throws Exception { - syncThread.halt(); - syncThread.join(0); - } - - protected final void connectToHdfs() throws IOException { - try { - HdfsResources resources = hdfsResources.get(); - - if (resources.getConfiguration() == null) { - resources = this.resetHDFSResources(hdfsSinkConfig); - hdfsResources.set(resources); - } - } catch (IOException ex) { - hdfsResources.set(new HdfsResources(null, null, null)); - throw ex; - } - } - - protected FSDataOutputStream getHdfsStream() throws IllegalArgumentException, IOException { - if (hdfsStream == null) { - Path path = getPath(); - FileSystem fs = getFileSystemAsUser(getConfiguration(), getUserGroupInformation()); - hdfsStream = fs.exists(path) ? fs.append(path) : fs.create(path); - } - return hdfsStream; - } - - protected final Path getPath() { - if (path == null) { - String ext = ""; - if (StringUtils.isNotBlank(hdfsSinkConfig.getFileExtension())) { - ext = hdfsSinkConfig.getFileExtension(); - } else if (getCompressionCodec() != null) { - ext = getCompressionCodec().getDefaultExtension(); - } - - String directory = hdfsSinkConfig.getDirectory(); - if (subdirectoryFormatter != null) { - directory = FilenameUtils.concat(directory, LocalDateTime.now().format(subdirectoryFormatter)); - } - path = new Path(FilenameUtils.concat(directory, - hdfsSinkConfig.getFilenamePrefix() + "-" + System.currentTimeMillis() + ext)); - log.info("Create path: {}", path); - } - return path; - } - - protected final void launchSyncThread() throws IOException { - syncThread = new HdfsSyncThread(getHdfsStream(), unackedRecords, hdfsSinkConfig.getSyncInterval()); - syncThread.start(); - } -} \ No newline at end of file diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsSinkConfig.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsSinkConfig.java deleted file mode 100644 index 9e1c6090fb5b7..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsSinkConfig.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2.sink; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; -import java.io.File; -import java.io.IOException; -import java.io.Serializable; -import java.time.LocalDateTime; -import java.time.format.DateTimeFormatter; -import java.util.Map; -import lombok.Data; -import lombok.EqualsAndHashCode; -import lombok.experimental.Accessors; -import org.apache.commons.lang.StringUtils; -import org.apache.pulsar.io.hdfs2.AbstractHdfsConfig; - -/** - * Configuration object for all HDFS Sink components. - */ -@Data -@EqualsAndHashCode(callSuper = false) -@Accessors(chain = true) -public class HdfsSinkConfig extends AbstractHdfsConfig implements Serializable { - - private static final long serialVersionUID = 1L; - - /** - * The prefix of the files to create inside the HDFS directory, i.e. a value of "topicA" - * will result in files named topicA-, topicA-, etc being produced - */ - private String filenamePrefix; - - /** - * The extension to add to the files written to HDFS, e.g. '.txt', '.seq', etc. - */ - private String fileExtension; - - /** - * The character to use to separate records in a text file. If no value is provided - * then the content from all of the records will be concatenated together in one continuous - * byte array. - */ - private char separator; - - /** - * The interval (in milliseconds) between calls to flush data to HDFS disk. - */ - private long syncInterval; - - /** - * The maximum number of records that we hold in memory before acking. Default is Integer.MAX_VALUE. - * Setting this value to one, results in every record being sent to disk before the record is acked, - * while setting it to a higher values allows us to buffer records before flushing them all to disk. - */ - private int maxPendingRecords = Integer.MAX_VALUE; - - /** - * A subdirectory associated with the created time of the sink. - * The pattern is the formatted pattern of {@link AbstractHdfsConfig#getDirectory()}'s subdirectory. - * - * @see java.time.format.DateTimeFormatter for pattern's syntax - */ - private String subdirectoryPattern; - - public static HdfsSinkConfig load(String yamlFile) throws IOException { - ObjectMapper mapper = new ObjectMapper(new YAMLFactory()); - return mapper.readValue(new File(yamlFile), HdfsSinkConfig.class); - } - - public static HdfsSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), HdfsSinkConfig.class); - } - - @Override - public void validate() { - super.validate(); - if ((StringUtils.isEmpty(fileExtension) && getCompression() == null) - || StringUtils.isEmpty(filenamePrefix)) { - throw new IllegalArgumentException("Required property not set."); - } - - if (syncInterval < 0) { - throw new IllegalArgumentException("Sync Interval cannot be negative"); - } - - if (maxPendingRecords < 1) { - throw new IllegalArgumentException("Max Pending Records must be a positive integer"); - } - - if (subdirectoryPattern != null) { - try { - LocalDateTime.of(2020, 1, 1, 12, 0).format(DateTimeFormatter.ofPattern(subdirectoryPattern)); - } catch (Exception e) { - throw new IllegalArgumentException(subdirectoryPattern + " is not a valid pattern: " + e.getMessage()); - } - } - } -} diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsSyncThread.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsSyncThread.java deleted file mode 100644 index 9ddd83f4423f9..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/HdfsSyncThread.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2.sink; - -import java.io.IOException; -import java.util.concurrent.BlockingQueue; -import org.apache.commons.collections4.CollectionUtils; -import org.apache.hadoop.fs.Syncable; -import org.apache.pulsar.functions.api.Record; - -/** - * A thread that runs in the background and acknowledges Records - * after they have been written to disk. - * - * @param - */ -public class HdfsSyncThread extends Thread { - - private final Syncable stream; - private final BlockingQueue> unackedRecords; - private final long syncInterval; - private boolean keepRunning = true; - - public HdfsSyncThread(Syncable stream, BlockingQueue> unackedRecords, long syncInterval) { - this.stream = stream; - this.unackedRecords = unackedRecords; - this.syncInterval = syncInterval; - } - - @Override - public void run() { - while (keepRunning) { - try { - Thread.sleep(syncInterval); - ackRecords(); - } catch (InterruptedException e) { - return; - } catch (IOException e) { - e.printStackTrace(); - } - } - } - - public final void halt() throws IOException, InterruptedException { - keepRunning = false; - ackRecords(); - } - - private void ackRecords() throws IOException, InterruptedException { - - if (CollectionUtils.isEmpty(unackedRecords)) { - return; - } - - synchronized (stream) { - stream.hsync(); - } - - while (!unackedRecords.isEmpty()) { - unackedRecords.take().ack(); - } - } -} diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/package-info.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/package-info.java deleted file mode 100644 index 238a441ee0ee3..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2.sink; \ No newline at end of file diff --git a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/seq/HdfsAbstractSequenceFileSink.java b/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/seq/HdfsAbstractSequenceFileSink.java deleted file mode 100644 index 355c00080effe..0000000000000 --- a/pulsar-io/hdfs2/src/main/java/org/apache/pulsar/io/hdfs2/sink/seq/HdfsAbstractSequenceFileSink.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.hdfs2.sink.seq; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.SequenceFile.Writer; -import org.apache.hadoop.io.SequenceFile.Writer.Option; -import org.apache.pulsar.functions.api.Record; -import org.apache.pulsar.io.core.KeyValue; -import org.apache.pulsar.io.core.Sink; -import org.apache.pulsar.io.hdfs2.sink.HdfsAbstractSink; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * HDFS Sink that writes it contents to HDFS as Sequence Files. - * - * @param - The incoming Key type - * @param - The incoming Value type - * @param - The HDFS Key type - * @param - The HDFS Value type - */ -public abstract class HdfsAbstractSequenceFileSink - extends HdfsAbstractSink implements Sink { - - private static final Logger LOG = LoggerFactory.getLogger(HdfsAbstractSequenceFileSink.class); - - protected AtomicLong counter; - protected FSDataOutputStream hdfsStream; - protected Writer writer = null; - - public abstract KeyValue convert(KeyValue kv); - - @Override - public void close() throws Exception { - writer.close(); - super.close(); - } - - @Override - protected void createWriter() throws IOException { - writer = getWriter(); - } - - @Override - public void write(Record record) { - try { - KeyValue kv = extractKeyValue(record); - KeyValue keyValue = convert(kv); - writer.append(keyValue.getKey(), keyValue.getValue()); - unackedRecords.put(record); - } catch (IOException | InterruptedException e) { - LOG.error("Unable to write to file " + getPath(), e); - record.fail(); - } - } - - protected Writer getWriter() throws IOException { - counter = new AtomicLong(0); - List

- * Example: - *

- * - *

-     * 
-     * ClientConfiguration confData = new ClientConfiguration();
-     * String authPluginClassName = "org.apache.pulsar.client.impl.auth.MyAuthentication";
-     * String authParamsString = "key1:val1,key2:val2";
-     * Authentication auth = AuthenticationFactory.create(authPluginClassName, authParamsString);
-     * confData.setAuthentication(auth);
-     * PulsarClient client = PulsarClient.create(serviceUrl, confData);
-     * ....
-     * 
-     * 
- * - * @param authentication - */ - public void setAuthentication(Authentication authentication) { - confData.setAuthentication(authentication); - } - - /** - * Set the authentication provider to use in the Pulsar client instance. - *

- * Example: - *

- * - *

-     * 
-     * ClientConfiguration confData = new ClientConfiguration();
-     * String authPluginClassName = "org.apache.pulsar.client.impl.auth.MyAuthentication";
-     * String authParamsString = "key1:val1,key2:val2";
-     * confData.setAuthentication(authPluginClassName, authParamsString);
-     * PulsarClient client = PulsarClient.create(serviceUrl, confData);
-     * ....
-     * 
-     * 
- * - * @param authPluginClassName - * name of the Authentication-Plugin you want to use - * @param authParamsString - * string which represents parameters for the Authentication-Plugin, e.g., "key1:val1,key2:val2" - * @throws UnsupportedAuthenticationException - * failed to instantiate specified Authentication-Plugin - */ - public void setAuthentication(String authPluginClassName, String authParamsString) - throws UnsupportedAuthenticationException { - confData.setAuthentication(AuthenticationFactory.create(authPluginClassName, authParamsString)); - } - - /** - * Set the authentication provider to use in the Pulsar client instance. - *

- * Example: - *

- * - *

-     * 
-     * ClientConfiguration confData = new ClientConfiguration();
-     * String authPluginClassName = "org.apache.pulsar.client.impl.auth.MyAuthentication";
-     * Map authParams = new HashMap();
-     * authParams.put("key1", "val1");
-     * confData.setAuthentication(authPluginClassName, authParams);
-     * PulsarClient client = PulsarClient.create(serviceUrl, confData);
-     * ....
-     * 
-     * 
- * - * @param authPluginClassName - * name of the Authentication-Plugin you want to use - * @param authParams - * map which represents parameters for the Authentication-Plugin - * @throws UnsupportedAuthenticationException - * failed to instantiate specified Authentication-Plugin - */ - public void setAuthentication(String authPluginClassName, Map authParams) - throws UnsupportedAuthenticationException { - confData.setAuthentication(AuthenticationFactory.create(authPluginClassName, authParams)); - } - - /** - * @return the operation timeout in ms - */ - public long getOperationTimeoutMs() { - return confData.getOperationTimeoutMs(); - } - - /** - * Set the operation timeout (default: 30 seconds). - *

- * Producer-create, subscribe and unsubscribe operations will be retried until this interval, after which the - * operation will be marked as failed - * - * @param operationTimeout - * operation timeout - * @param unit - * time unit for {@code operationTimeout} - */ - public void setOperationTimeout(int operationTimeout, TimeUnit unit) { - checkArgument(operationTimeout >= 0); - confData.setOperationTimeoutMs(unit.toMillis(operationTimeout)); - } - - /** - * @return the number of threads to use for handling connections - */ - public int getIoThreads() { - return confData.getNumIoThreads(); - } - - /** - * Set the number of threads to be used for handling connections to brokers (default: 1 thread). - * - * @param numIoThreads - */ - public void setIoThreads(int numIoThreads) { - checkArgument(numIoThreads > 0); - confData.setNumIoThreads(numIoThreads); - } - - /** - * @return the number of threads to use for message listeners - */ - public int getListenerThreads() { - return confData.getNumListenerThreads(); - } - - /** - * Set the number of threads to be used for message listeners (default: 1 thread). - * - * @param numListenerThreads - */ - public void setListenerThreads(int numListenerThreads) { - checkArgument(numListenerThreads > 0); - confData.setNumListenerThreads(numListenerThreads); - } - - /** - * @return the max number of connections per single broker - */ - public int getConnectionsPerBroker() { - return confData.getConnectionsPerBroker(); - } - - /** - * Sets the max number of connection that the client library will open to a single broker. - *

- * By default, the connection pool will use a single connection for all the producers and consumers. Increasing this - * parameter may improve throughput when using many producers over a high latency connection. - *

- * - * @param connectionsPerBroker - * max number of connections per broker (needs to be greater than 0) - */ - public void setConnectionsPerBroker(int connectionsPerBroker) { - checkArgument(connectionsPerBroker > 0, "Connections per broker need to be greater than 0"); - confData.setConnectionsPerBroker(connectionsPerBroker); - } - - /** - * @return whether TCP no-delay should be set on the connections - */ - public boolean isUseTcpNoDelay() { - return confData.isUseTcpNoDelay(); - } - - /** - * Configure whether to use TCP no-delay flag on the connection, to disable Nagle algorithm. - *

- * No-delay features make sure packets are sent out on the network as soon as possible, and it's critical to achieve - * low latency publishes. On the other hand, sending out a huge number of small packets might limit the overall - * throughput, so if latency is not a concern, it's advisable to set the useTcpNoDelay flag to false. - *

- * Default value is true - * - * @param useTcpNoDelay - */ - public void setUseTcpNoDelay(boolean useTcpNoDelay) { - confData.setUseTcpNoDelay(useTcpNoDelay); - } - - /** - * @return whether TLS encryption is used on the connection - */ - public boolean isUseTls() { - return confData.isUseTls(); - } - - /** - * Configure whether to use TLS encryption on the connection (default: false). - * - * @param useTls - */ - public void setUseTls(boolean useTls) { - confData.setUseTls(useTls); - } - - /** - * @return path to the trusted TLS certificate file - */ - public String getTlsTrustCertsFilePath() { - return confData.getTlsTrustCertsFilePath(); - } - - /** - * Set the path to the trusted TLS certificate file. - * - * @param tlsTrustCertsFilePath - */ - public void setTlsTrustCertsFilePath(String tlsTrustCertsFilePath) { - confData.setTlsTrustCertsFilePath(tlsTrustCertsFilePath); - } - - /** - * @return whether the Pulsar client accept untrusted TLS certificate from broker - */ - public boolean isTlsAllowInsecureConnection() { - return confData.isTlsAllowInsecureConnection(); - } - - /** - * Configure whether the Pulsar client accept untrusted TLS certificate from broker (default: false). - * - * @param tlsAllowInsecureConnection - */ - public void setTlsAllowInsecureConnection(boolean tlsAllowInsecureConnection) { - confData.setTlsAllowInsecureConnection(tlsAllowInsecureConnection); - } - - /** - * Stats will be activated with positive statsIntervalSeconds. - * - * @return the interval between each stat info (default: 60 seconds) - */ - public long getStatsIntervalSeconds() { - return confData.getStatsIntervalSeconds(); - } - - /** - * Set the interval between each stat info (default: 60 seconds) Stats will be activated with positive. - * statsIntervalSeconds It should be set to at least 1 second - * - * @param statsInterval - * the interval between each stat info - * @param unit - * time unit for {@code statsInterval} - */ - public void setStatsInterval(long statsInterval, TimeUnit unit) { - confData.setStatsIntervalSeconds(unit.toSeconds(statsInterval)); - } - - /** - * Get configured total allowed concurrent lookup-request. - * - * @return - */ - public int getConcurrentLookupRequest() { - return confData.getConcurrentLookupRequest(); - } - - /** - * Number of concurrent lookup-requests allowed on each broker-connection to prevent overload on broker. - * (default: 50000) It should be configured with higher value only in case of it requires to - * produce/subscribe on thousands of topic using created {@link PulsarClient} - * - * @param concurrentLookupRequest - */ - public void setConcurrentLookupRequest(int concurrentLookupRequest) { - confData.setConcurrentLookupRequest(concurrentLookupRequest); - } - - /** - * Get configured max number of reject-request in a time-frame (60 seconds) after which connection will be closed. - * - * @return - */ - public int getMaxNumberOfRejectedRequestPerConnection() { - return confData.getMaxNumberOfRejectedRequestPerConnection(); - } - - /** - * Set max number of broker-rejected requests in a certain time-frame (60 seconds) after which current connection. - * will be closed and client creates a new connection that give chance to connect a different broker (default: - * 50) - * - * @param maxNumberOfRejectedRequestPerConnection - */ - public void setMaxNumberOfRejectedRequestPerConnection(int maxNumberOfRejectedRequestPerConnection) { - confData.setMaxNumberOfRejectedRequestPerConnection(maxNumberOfRejectedRequestPerConnection); - } - - public boolean isTlsHostnameVerificationEnable() { - return confData.isTlsHostnameVerificationEnable(); - } - - /** - * It allows to validate hostname verification when client connects to broker over tls. It validates incoming x509 - * certificate and matches provided hostname(CN/SAN) with expected broker's host name. It follows RFC 2818, 3.1. - * Server Identity hostname verification. - * - * @see rfc2818 - * - * @param tlsHostnameVerificationEnable - */ - public void setTlsHostnameVerificationEnable(boolean tlsHostnameVerificationEnable) { - confData.setTlsHostnameVerificationEnable(tlsHostnameVerificationEnable); - } - - public ClientConfiguration setServiceUrl(String serviceUrl) { - confData.setServiceUrl(serviceUrl); - return this; - } - - /** - * Set the duration of time to wait for a connection to a broker to be established. If the duration - * passes without a response from the broker, the connection attempt is dropped. - * - * @param duration the duration to wait - * @param unit the time unit in which the duration is defined - */ - public void setConnectionTimeout(int duration, TimeUnit unit) { - confData.setConnectionTimeoutMs((int) unit.toMillis(duration)); - } - - /** - * Get the duration of time for which the client will wait for a connection to a broker to be - * established before giving up. - * - * @return the duration, in milliseconds - */ - public long getConnectionTimeoutMs() { - return confData.getConnectionTimeoutMs(); - } - - public ClientConfigurationData getConfigurationData() { - return confData; - } - -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/Consumer.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/Consumer.java deleted file mode 100644 index d84b9981b0320..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/Consumer.java +++ /dev/null @@ -1,331 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import java.io.Closeable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; - -/** - * An interface that abstracts behavior of Pulsar's consumer. - */ -public interface Consumer extends Closeable { - - /** - * Get a topic for the consumer. - * - * @return topic for the consumer - */ - String getTopic(); - - /** - * Get a subscription for the consumer. - * - * @return subscription for the consumer - */ - String getSubscription(); - - /** - * Unsubscribe the consumer - *

- * This call blocks until the consumer is unsubscribed. - * - * @throws PulsarClientException - */ - void unsubscribe() throws PulsarClientException; - - /** - * Asynchronously unsubscribe the consumer. - * - * @return {@link CompletableFuture} for this operation - */ - CompletableFuture unsubscribeAsync(); - - /** - * Receives a single message. - *

- * This calls blocks until a message is available. - * - * @return the received message - * @throws PulsarClientException.AlreadyClosedException - * if the consumer was already closed - * @throws PulsarClientException.InvalidConfigurationException - * if a message listener was defined in the configuration - */ - Message receive() throws PulsarClientException; - - /** - * Receive a single message - *

- * Retrieves a message when it will be available and completes {@link CompletableFuture} with received message. - *

- *

- * {@code receiveAsync()} should be called subsequently once returned {@code CompletableFuture} gets complete with - * received message. Else it creates backlog of receive requests in the application. - *

- * - * @return {@link CompletableFuture}<{@link Message}> will be completed when message is available - */ - CompletableFuture> receiveAsync(); - - /** - * Receive a single message - *

- * Retrieves a message, waiting up to the specified wait time if necessary. - * - * @param timeout - * 0 or less means immediate rather than infinite - * @param unit - * @return the received {@link Message} or null if no message available before timeout - * @throws PulsarClientException.AlreadyClosedException - * if the consumer was already closed - * @throws PulsarClientException.InvalidConfigurationException - * if a message listener was defined in the configuration - */ - Message receive(int timeout, TimeUnit unit) throws PulsarClientException; - - /** - * Acknowledge the consumption of a single message. - * - * @param message - * The {@code Message} to be acknowledged - * @throws PulsarClientException.AlreadyClosedException - * if the consumer was already closed - */ - void acknowledge(Message message) throws PulsarClientException; - - /** - * Acknowledge the consumption of a single message, identified by its MessageId. - * - * @param messageId - * The {@code MessageId} to be acknowledged - * @throws PulsarClientException.AlreadyClosedException - * if the consumer was already closed - */ - void acknowledge(MessageId messageId) throws PulsarClientException; - - /** - * Acknowledge the reception of all the messages in the stream up to (and including) the provided message. - * - * This method will block until the acknowledge has been sent to the broker. After that, the messages will not be - * re-delivered to this consumer. - * - * Cumulative acknowledge cannot be used when the consumer type is set to ConsumerShared. - * - * It's equivalent to calling asyncAcknowledgeCumulative(Message) and waiting for the callback to be triggered. - * - * @param message - * The {@code Message} to be cumulatively acknowledged - * @throws PulsarClientException.AlreadyClosedException - * if the consumer was already closed - */ - void acknowledgeCumulative(Message message) throws PulsarClientException; - - /** - * Acknowledge the reception of all the messages in the stream up to (and including) the provided message. - * - * This method will block until the acknowledge has been sent to the broker. After that, the messages will not be - * re-delivered to this consumer. - * - * Cumulative acknowledge cannot be used when the consumer type is set to ConsumerShared. - * - * It's equivalent to calling asyncAcknowledgeCumulative(MessageId) and waiting for the callback to be triggered. - * - * @param messageId - * The {@code MessageId} to be cumulatively acknowledged - * @throws PulsarClientException.AlreadyClosedException - * if the consumer was already closed - */ - void acknowledgeCumulative(MessageId messageId) throws PulsarClientException; - - /** - * Asynchronously acknowledge the consumption of a single message. - * - * @param message - * The {@code Message} to be acknowledged - * @return a future that can be used to track the completion of the operation - */ - CompletableFuture acknowledgeAsync(Message message); - - /** - * Asynchronously acknowledge the consumption of a single message. - * - * @param messageId - * The {@code MessageId} to be acknowledged - * @return a future that can be used to track the completion of the operation - */ - CompletableFuture acknowledgeAsync(MessageId messageId); - - /** - * Asynchronously Acknowledge the reception of all the messages in the stream up to (and including) the provided - * message. - * - * Cumulative acknowledge cannot be used when the consumer type is set to ConsumerShared. - * - * @param message - * The {@code Message} to be cumulatively acknowledged - * @return a future that can be used to track the completion of the operation - */ - CompletableFuture acknowledgeCumulativeAsync(Message message); - - /** - * Asynchronously Acknowledge the reception of all the messages in the stream up to (and including) the provided - * message. - * - * Cumulative acknowledge cannot be used when the consumer type is set to ConsumerShared. - * - * @param messageId - * The {@code MessageId} to be cumulatively acknowledged - * @return a future that can be used to track the completion of the operation - */ - CompletableFuture acknowledgeCumulativeAsync(MessageId messageId); - - /** - * Get statistics for the consumer. - * - *

    - *
  • numMsgsReceived : Number of messages received in the current interval - *
  • numBytesReceived : Number of bytes received in the current interval - *
  • numReceiveFailed : Number of messages failed to receive in the current interval - *
  • numAcksSent : Number of acks sent in the current interval - *
  • numAcksFailed : Number of acks failed to send in the current interval - *
  • totalMsgsReceived : Total number of messages received - *
  • totalBytesReceived : Total number of bytes received - *
  • totalReceiveFailed : Total number of messages failed to receive - *
  • totalAcksSent : Total number of acks sent - *
  • totalAcksFailed : Total number of acks failed to sent - *
- * - * @return statistic for the consumer - */ - ConsumerStats getStats(); - - /** - * Close the consumer and stop the broker to push more messages. - */ - @Override - void close() throws PulsarClientException; - - /** - * Asynchronously close the consumer and stop the broker to push more messages. - * - * @return a future that can be used to track the completion of the operation - */ - CompletableFuture closeAsync(); - - /** - * Return true if the topic was terminated and this consumer has already consumed all the messages in the topic. - * - * Please note that this does not simply mean that the consumer is caught up with the last message published by - * producers, rather the topic needs to be explicitly "terminated". - */ - boolean hasReachedEndOfTopic(); - - /** - * Redelivers all the unacknowledged messages. In Failover mode, the request is ignored if the consumer is not - * active for the given topic. In Shared mode, the consumers messages to be redelivered are distributed across all - * the connected consumers. This is a non blocking call and doesn't throw an exception. In case the connection - * breaks, the messages are redelivered after reconnect. - */ - void redeliverUnacknowledgedMessages(); - - /** - * Reset the subscription associated with this consumer to a specific message id. - *

- * - * The message id can either be a specific message or represent the first or last messages in the topic. - *

- *

    - *
  • MessageId.earliest : Reset the subscription on the earliest message available in the topic - *
  • MessageId.latest : Reset the subscription on the latest message in the topic - *
- * - * Note: this operation can only be done on non-partitioned topics. For these, one can rather perform the seek() on - * the individual partitions. - * - * @param messageId - * the message id where to reposition the subscription - */ - void seek(MessageId messageId) throws PulsarClientException; - - /** - * Reset the subscription associated with this consumer to a specific message publish time. - * - * Note: this operation can only be done on non-partitioned topics. For these, one can rather perform the seek() on - * the individual partitions. - * - * @param timestamp - * the message publish time where to reposition the subscription - */ - void seek(long timestamp) throws PulsarClientException; - - /** - * Reset the subscription associated with this consumer to a specific message id. - *

- * - * The message id can either be a specific message or represent the first or last messages in the topic. - *

- *

    - *
  • MessageId.earliest : Reset the subscription on the earliest message available in the topic - *
  • MessageId.latest : Reset the subscription on the latest message in the topic - *
- * - * Note: this operation can only be done on non-partitioned topics. For these, one can rather perform the seek() on - * the individual partitions. - * - * @param messageId - * the message id where to reposition the subscription - * @return a future to track the completion of the seek operation - */ - CompletableFuture seekAsync(MessageId messageId); - - /** - * Reset the subscription associated with this consumer to a specific message publish time. - * - * Note: this operation can only be done on non-partitioned topics. For these, one can rather perform the seek() on - * the individual partitions. - * - * @param timestamp - * the message publish time where to reposition the subscription - * @return a future to track the completion of the seek operation - */ - CompletableFuture seekAsync(long timestamp); - - /** - * @return Whether the consumer is connected to the broker - */ - boolean isConnected(); - - /** - * Get the name of consumer. - * @return consumer name. - */ - String getConsumerName(); - - /** - * Stop requesting new messages from the broker until {@link #resume()} is called. Note that this might cause - * {@link #receive()} to block until {@link #resume()} is called and new messages are pushed by the broker. - */ - void pause(); - - /** - * Resume requesting messages from the broker. - */ - void resume(); -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java deleted file mode 100644 index 81956db56f774..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ConsumerConfiguration.java +++ /dev/null @@ -1,411 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import static com.google.common.base.Preconditions.checkArgument; -import java.io.Serializable; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.TimeUnit; -import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; -import org.apache.pulsar.client.impl.v1.ConsumerV1Impl; -/** - * Class specifying the configuration of a consumer. In Exclusive subscription, only a single consumer is allowed to - * attach to the subscription. Other consumers will get an error message. In Shared subscription, multiple consumers - * will be able to use the same subscription name and the messages will be dispatched in a round robin fashion. - * - * @deprecated Use {@link PulsarClient#newConsumer} to build and configure a {@link Consumer} instance - */ -@Deprecated -public class ConsumerConfiguration implements Serializable { - - /** - * Resend shouldn't be requested before minAckTimeoutMillis. - */ - static long minAckTimeoutMillis = 1000; - - private static final long serialVersionUID = 1L; - - private final ConsumerConfigurationData conf = new ConsumerConfigurationData<>(); - - private MessageListener messageListener; - - public ConsumerConfiguration() { - // Disable acknowledgment grouping when using v1 API - conf.setAcknowledgementsGroupTimeMicros(0); - } - - /** - * @return the configured timeout in milliseconds for unacked messages. - */ - public long getAckTimeoutMillis() { - return conf.getAckTimeoutMillis(); - } - - /** - * Set the timeout for unacked messages, truncated to the nearest millisecond. The timeout needs to be greater than - * 10 seconds. - * - * @param ackTimeout - * for unacked messages. - * @param timeUnit - * unit in which the timeout is provided. - * @return {@link ConsumerConfiguration} - */ - public ConsumerConfiguration setAckTimeout(long ackTimeout, TimeUnit timeUnit) { - long ackTimeoutMillis = timeUnit.toMillis(ackTimeout); - checkArgument(ackTimeoutMillis >= minAckTimeoutMillis, - "Ack timeout should be should be greater than " + minAckTimeoutMillis + " ms"); - conf.setAckTimeoutMillis(timeUnit.toMillis(ackTimeout)); - return this; - } - - /** - * @return the configured subscription type - */ - public SubscriptionType getSubscriptionType() { - return conf.getSubscriptionType(); - } - - /** - * Select the subscription type to be used when subscribing to the topic. - *

- * Default is {@link SubscriptionType#Exclusive} - * - * @param subscriptionType - * the subscription type value - */ - public ConsumerConfiguration setSubscriptionType(SubscriptionType subscriptionType) { - Objects.requireNonNull(subscriptionType); - conf.setSubscriptionType(subscriptionType); - return this; - } - - /** - * @return the configured {@link MessageListener} for the consumer - */ - public MessageListener getMessageListener() { - return messageListener; - } - - /** - * Sets a {@link MessageListener} for the consumer - *

- * When a {@link MessageListener} is set, application will receive messages through it. Calls to - * {@link Consumer#receive()} will not be allowed. - * - * @param messageListener - * the listener object - */ - public ConsumerConfiguration setMessageListener(MessageListener messageListener) { - Objects.requireNonNull(messageListener); - this.messageListener = messageListener; - conf.setMessageListener(new org.apache.pulsar.shade.client.api.v2.MessageListener() { - - @Override - public void received(org.apache.pulsar.shade.client.api.v2.Consumer consumer, Message msg) { - messageListener.received(new ConsumerV1Impl(consumer), msg); - } - - @Override - public void reachedEndOfTopic(org.apache.pulsar.shade.client.api.v2.Consumer consumer) { - messageListener.reachedEndOfTopic(new ConsumerV1Impl(consumer)); - } - }); - return this; - } - - /** - * @return this configured {@link ConsumerEventListener} for the consumer. - * @see #setConsumerEventListener(ConsumerEventListener) - * @since 2.0 - */ - public ConsumerEventListener getConsumerEventListener() { - return conf.getConsumerEventListener(); - } - - /** - * Sets a {@link ConsumerEventListener} for the consumer. - * - *

- * The consumer group listener is used for receiving consumer state change in a consumer group for failover - * subscription. Application can then react to the consumer state changes. - * - *

- * This change is experimental. It is subject to changes coming in release 2.0. - * - * @param listener - * the consumer group listener object - * @return consumer configuration - * @since 2.0 - */ - public ConsumerConfiguration setConsumerEventListener(ConsumerEventListener listener) { - Objects.requireNonNull(listener); - conf.setConsumerEventListener(listener); - return this; - } - - /** - * @return the configure receiver queue size value - */ - public int getReceiverQueueSize() { - return conf.getReceiverQueueSize(); - } - - /** - * @return the configured max total receiver queue size across partitions - */ - public int getMaxTotalReceiverQueueSizeAcrossPartitions() { - return conf.getMaxTotalReceiverQueueSizeAcrossPartitions(); - } - - /** - * Set the max total receiver queue size across partitons. - *

- * This setting will be used to reduce the receiver queue size for individual partitions - * {@link #setReceiverQueueSize(int)} if the total exceeds this value (default: 50000). - * - * @param maxTotalReceiverQueueSizeAcrossPartitions - */ - public void setMaxTotalReceiverQueueSizeAcrossPartitions(int maxTotalReceiverQueueSizeAcrossPartitions) { - checkArgument(maxTotalReceiverQueueSizeAcrossPartitions >= conf.getReceiverQueueSize()); - conf.setMaxTotalReceiverQueueSizeAcrossPartitions(maxTotalReceiverQueueSizeAcrossPartitions); - } - - /** - * @return the CryptoKeyReader - */ - public CryptoKeyReader getCryptoKeyReader() { - return conf.getCryptoKeyReader(); - } - - /** - * Sets a {@link CryptoKeyReader}. - * - * @param cryptoKeyReader - * CryptoKeyReader object - */ - public ConsumerConfiguration setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { - Objects.requireNonNull(cryptoKeyReader); - conf.setCryptoKeyReader(cryptoKeyReader); - return this; - } - - /** - * Sets the ConsumerCryptoFailureAction to the value specified. - * - * @param action - * consumer action - */ - public void setCryptoFailureAction(ConsumerCryptoFailureAction action) { - conf.setCryptoFailureAction(action); - } - - /** - * @return The ConsumerCryptoFailureAction - */ - public ConsumerCryptoFailureAction getCryptoFailureAction() { - return conf.getCryptoFailureAction(); - } - - /** - * Sets the size of the consumer receive queue. - *

- * The consumer receive queue controls how many messages can be accumulated by the {@link Consumer} before the - * application calls {@link Consumer#receive()}. Using a higher value could potentially increase the consumer - * throughput at the expense of bigger memory utilization. - *

- *

- * Setting the consumer queue size as zero - *

    - *
  • Decreases the throughput of the consumer, by disabling pre-fetching of messages. This approach improves the - * message distribution on shared subscription, by pushing messages only to the consumers that are ready to process - * them. Neither {@link Consumer#receive(int, TimeUnit)} nor Partitioned Topics can be used if the consumer queue - * size is zero. {@link Consumer#receive()} function call should not be interrupted when the consumer queue size is - * zero.
  • - *
  • Doesn't support Batch-Message: if consumer receives any batch-message then it closes consumer connection with - * broker and {@link Consumer#receive()} call will remain blocked while {@link Consumer#receiveAsync()} receives - * exception in callback. consumer will not be able receive any further message unless batch-message in pipeline - * is removed
  • - *
- *

- * Default value is {@code 1000} messages and should be good for most use cases. - * - * @param receiverQueueSize - * the new receiver queue size value - */ - public ConsumerConfiguration setReceiverQueueSize(int receiverQueueSize) { - checkArgument(receiverQueueSize >= 0, "Receiver queue size cannot be negative"); - conf.setReceiverQueueSize(receiverQueueSize); - return this; - } - - /** - * @return the consumer name - */ - public String getConsumerName() { - return conf.getConsumerName(); - } - - /** - * Set the consumer name. - * - * @param consumerName - */ - public ConsumerConfiguration setConsumerName(String consumerName) { - checkArgument(consumerName != null && !consumerName.equals("")); - conf.setConsumerName(consumerName); - return this; - } - - public int getPriorityLevel() { - return conf.getPriorityLevel(); - } - - /** - * Sets priority level for the shared subscription consumers to which broker gives more priority while dispatching - * messages. Here, broker follows descending priorities. (eg: 0=max-priority, 1, 2,..)
- * In Shared subscription mode, broker will first dispatch messages to max priority-level consumers if they have - * permits, else broker will consider next priority level consumers.
- * If subscription has consumer-A with priorityLevel 0 and Consumer-B with priorityLevel 1 then broker will dispatch - * messages to only consumer-A until it runs out permit and then broker starts dispatching messages to Consumer-B. - * - *
-     * Consumer PriorityLevel Permits
-     * C1       0             2
-     * C2       0             1
-     * C3       0             1
-     * C4       1             2
-     * C5       1             1
-     * Order in which broker dispatches messages to consumers: C1, C2, C3, C1, C4, C5, C4
-     * 
- * - * @param priorityLevel - */ - public void setPriorityLevel(int priorityLevel) { - conf.setPriorityLevel(priorityLevel); - } - - public boolean getReadCompacted() { - return conf.isReadCompacted(); - } - - /** - * If enabled, the consumer will read messages from the compacted topic rather than reading the full message backlog - * of the topic. This means that, if the topic has been compacted, the consumer will only see the latest value for - * each key in the topic, up until the point in the topic message backlog that has been compacted. Beyond that - * point, the messages will be sent as normal. - * - * readCompacted can only be enabled subscriptions to persistent topics, which have a single active consumer (i.e. - * failure or exclusive subscriptions). Attempting to enable it on subscriptions to a non-persistent topics or on a - * shared subscription, will lead to the subscription call throwing a PulsarClientException. - * - * @param readCompacted - * whether to read from the compacted topic - */ - public ConsumerConfiguration setReadCompacted(boolean readCompacted) { - conf.setReadCompacted(readCompacted); - return this; - } - - /** - * Set a name/value property with this consumer. - * - * @param key - * @param value - * @return - */ - public ConsumerConfiguration setProperty(String key, String value) { - checkArgument(key != null); - checkArgument(value != null); - conf.getProperties().put(key, value); - return this; - } - - /** - * Add all the properties in the provided map. - * - * @param properties - * @return - */ - public ConsumerConfiguration setProperties(Map properties) { - conf.getProperties().putAll(properties); - return this; - } - - public Map getProperties() { - return conf.getProperties(); - } - - public ConsumerConfigurationData getConfigurationData() { - return conf; - } - - /** - * @param subscriptionInitialPosition the initial position at which to set - * set cursor when subscribing to the topic first time - * Default is {@value InitialPosition.Latest} - */ - public ConsumerConfiguration setSubscriptionInitialPosition( - SubscriptionInitialPosition subscriptionInitialPosition) { - conf.setSubscriptionInitialPosition(subscriptionInitialPosition); - return this; - } - - /** - * @return the configured {@link subscriptionInitialPosition} for the consumer - */ - public SubscriptionInitialPosition getSubscriptionInitialPosition(){ - return conf.getSubscriptionInitialPosition(); - } - - /** - * @return the configured {@link RedeliveryBackoff} for the consumer - */ - public RedeliveryBackoff getNegativeAckRedeliveryBackoff() { - return conf.getNegativeAckRedeliveryBackoff(); - } - - /** - * @param negativeAckRedeliveryBackoff the negative ack redelivery backoff policy. - * Default value is: MultiplierRedeliveryBackoff - * @return the {@link ConsumerConfiguration} - */ - public ConsumerConfiguration setNegativeAckRedeliveryBackoff(RedeliveryBackoff negativeAckRedeliveryBackoff) { - conf.setNegativeAckRedeliveryBackoff(negativeAckRedeliveryBackoff); - return this; - } - - /** - * @return the configured {@link RedeliveryBackoff} for the consumer - */ - public RedeliveryBackoff getAckTimeoutRedeliveryBackoff() { - return conf.getAckTimeoutRedeliveryBackoff(); - } - - /** - * @param ackTimeoutRedeliveryBackoff redelivery backoff policy for ack timeout. - * Default value is: MultiplierRedeliveryBackoff - * @return the {@link ConsumerConfiguration} - */ - public ConsumerConfiguration setAckTimeoutRedeliveryBackoff(RedeliveryBackoff ackTimeoutRedeliveryBackoff) { - conf.setAckTimeoutRedeliveryBackoff(ackTimeoutRedeliveryBackoff); - return this; - } -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/MessageBuilder.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/MessageBuilder.java deleted file mode 100644 index 084312ed28c07..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/MessageBuilder.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Map; -import org.apache.pulsar.client.impl.MessageBuilderImpl; - -/** - * Message builder factory. Use this class to create messages to be send to the Pulsar producer - * - * @deprecated since 2.0. Use {@link TypedMessageBuilder} as returned by {@link Producer#newMessage()} to create a new - * message builder. - */ -@Deprecated -public interface MessageBuilder { - - static MessageBuilder create() { - return new MessageBuilderImpl(); - } - - /** - * Finalize the immutable message. - * - * @return a {@link Message} ready to be sent through a {@link Producer} - */ - Message build(); - - /** - * Set the content of the message. - * - * @param data - * array containing the payload - */ - MessageBuilder setContent(byte[] data); - - /** - * Set the content of the message. - * - * @param data - * array containing the payload - * @param offset - * offset into the data array - * @param length - * length of the payload starting from the above offset - */ - MessageBuilder setContent(byte[] data, int offset, int length); - - /** - * Set the content of the message. - * - * @param buf - * a {@link ByteBuffer} with the payload of the message - */ - MessageBuilder setContent(ByteBuffer buf); - - /** - * Sets a new property on a message. - * - * @param name - * the name of the property - * @param value - * the associated value - */ - MessageBuilder setProperty(String name, String value); - - /** - * Add all the properties in the provided map. - */ - MessageBuilder setProperties(Map properties); - - /** - * Sets the key of the message for routing policy. - * - * @param key - */ - MessageBuilder setKey(String key); - - /** - * Set the event time for a given message. - * - *

- * Applications can retrieve the event time by calling {@link Message#getEventTime()}. - * - *

- * Note: currently pulsar doesn't support event-time based index. so the subscribers can't seek the messages by - * event time. - * - * @since 1.20.0 - */ - MessageBuilder setEventTime(long timestamp); - - /** - * Specify a custom sequence id for the message being published. - *

- * The sequence id can be used for deduplication purposes and it needs to follow these rules: - *

    - *
  1. sequenceId >= 0 - *
  2. Sequence id for a message needs to be greater than sequence id for earlier messages: - * sequenceId(N+1) > sequenceId(N) - *
  3. It's not necessary for sequence ids to be consecutive. There can be holes between messages. Eg. the - * sequenceId could represent an offset or a cumulative size. - *
- * - * @param sequenceId - * the sequence id to assign to the current message - * @since 1.20.0 - */ - MessageBuilder setSequenceId(long sequenceId); - - /** - * Override the replication clusters for this message. - * - * @param clusters - */ - MessageBuilder setReplicationClusters(List clusters); - - /** - * Disable replication for this message. - */ - MessageBuilder disableReplication(); -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/MessageListener.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/MessageListener.java deleted file mode 100644 index 301740be398c0..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/MessageListener.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import java.io.Serializable; - -/** - * A listener that will be called in order for every message received. - * - * - */ -public interface MessageListener extends Serializable { - /** - * This method is called whenever a new message is received. - * - * Messages are guaranteed to be delivered in order and from the same thread for a single consumer - * - * This method will only be called once for each message, unless either application or broker crashes. - * - * Implementation should acknowledge messages by calling consumer.acknowledge(msg). - * - * Application is responsible of handling any exception that could be thrown while processing the message. - * - * @param consumer - * the consumer that received the message - * @param msg - * the message object - */ - void received(Consumer consumer, Message msg); - - /** - * Get the notification when a topic is terminated. - * - * @param consumer - * the Consumer object associated with the terminated topic - */ - default void reachedEndOfTopic(Consumer consumer) { - // By default ignore the notification - } -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/Producer.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/Producer.java deleted file mode 100644 index 0b431050377c3..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/Producer.java +++ /dev/null @@ -1,199 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import java.io.Closeable; -import java.util.concurrent.CompletableFuture; - -/** - * Producer object. - * - * The producer is used to publish messages on a topic - * - * - */ -public interface Producer extends Closeable { - - /** - * @return the topic which producer is publishing to - */ - String getTopic(); - - /** - * @return the producer name which could have been assigned by the system or specified by the client - */ - String getProducerName(); - - /** - * Sends a message. - *

- * This call will be blocking until is successfully acknowledged by the Pulsar broker. - *

- * Use {@link #newMessage()} to specify more properties than just the value on the message to be sent. - * - * @param message - * a message - * @return the message id assigned to the published message - * @throws PulsarClientException.TimeoutException - * if the message was not correctly received by the system within the timeout period - * @throws PulsarClientException.AlreadyClosedException - * if the producer was already closed - */ - MessageId send(byte[] message) throws PulsarClientException; - - /** - * Send a message asynchronously - *

- * When the producer queue is full, by default this method will complete the future with an exception - * {@link PulsarClientException.ProducerQueueIsFullError} - *

- * See {@link ProducerBuilder#maxPendingMessages(int)} to configure the producer queue size and - * {@link ProducerBuilder#blockIfQueueFull(boolean)} to change the blocking behavior. - *

- * Use {@link #newMessage()} to specify more properties than just the value on the message to be sent. - * - * @param message - * a byte array with the payload of the message - * @return a future that can be used to track when the message will have been safely persisted - */ - CompletableFuture sendAsync(byte[] message); - - /** - * Flush all the messages buffered in the client and wait until all messages have been successfully persisted. - * - * @throws PulsarClientException - * @since 2.1.0 - * @see #flushAsync() - */ - void flush() throws PulsarClientException; - - /** - * Flush all the messages buffered in the client asynchronously. - * - * @return a future that can be used to track when all the messages have been safely persisted. - * @since 2.1.0 - * @see #flush() - */ - CompletableFuture flushAsync(); - - /** - * Send a message. - * - * @param message - * a message - * @return the message id assigned to the published message - * @throws PulsarClientException.TimeoutException - * if the message was not correctly received by the system within the timeout period - * - * @deprecated since 2.0. Use {@link TypedMessageBuilder} as returned by {@link Producer.newMessage()} to create a - * new message builder. - */ - @Deprecated - MessageId send(Message message) throws PulsarClientException; - - /** - * Send a message asynchronously. - *

- * When the returned {@link CompletableFuture} is marked as completed successfully, the provided message will - * contain the {@link MessageId} assigned by the broker to the published message. - *

- * Example: - * - *

-     * Message msg = MessageBuilder.create().setContent(myContent).build();
-     * producer.sendAsync(msg).thenRun(v -> {
-     *    System.out.println("Published message: " + msg.getMessageId());
-     * }).exceptionally(e -> {
-     *    // Failed to publish
-     * });
-     * 
- *

- * When the producer queue is full, by default this method will complete the future with an exception - * {@link PulsarClientException.ProducerQueueIsFullError} - *

- * See {@link ProducerBuilder#maxPendingMessages(int)} to configure the producer queue size and - * {@link ProducerBuilder#blockIfQueueFull(boolean)} to change the blocking behavior. - * - * @param message - * a message - * @return a future that can be used to track when the message will have been safely persisted - * @deprecated since 2.0. Use {@link TypedMessageBuilder} as returned by {@link Producer#newMessage()} to create a - * new message builder. - */ - @Deprecated - CompletableFuture sendAsync(Message message); - - /** - * Get the last sequence id that was published by this producer. - *

- * This represent either the automatically assigned or custom sequence id (set on the {@link MessageBuilder}) that - * was published and acknowledged by the broker. - *

- * After recreating a producer with the same producer name, this will return the last message that was published in - * the previous producer session, or -1 if there no message was ever published. - * - * @return the last sequence id published by this producer - */ - long getLastSequenceId(); - - /** - * Get statistics for the producer. - * - *

    - *
  • numMsgsSent : Number of messages sent in the current interval - *
  • numBytesSent : Number of bytes sent in the current interval - *
  • numSendFailed : Number of messages failed to send in the current interval - *
  • numAcksReceived : Number of acks received in the current interval - *
  • totalMsgsSent : Total number of messages sent - *
  • totalBytesSent : Total number of bytes sent - *
  • totalSendFailed : Total number of messages failed to send - *
  • totalAcksReceived: Total number of acks received - *
- * - * @return statistic for the producer or null if ProducerStatsRecorderImpl is disabled. - */ - ProducerStats getStats(); - - /** - * Close the producer and releases resources allocated. - * - * No more writes will be accepted from this producer. Waits until all pending write request are persisted. In case - * of errors, pending writes will not be retried. - * - * @throws PulsarClientException.AlreadyClosedException - * if the producer was already closed - */ - @Override - void close() throws PulsarClientException; - - /** - * Close the producer and releases resources allocated. - * - * No more writes will be accepted from this producer. Waits until all pending write request are persisted. In case - * of errors, pending writes will not be retried. - * - * @return a future that can used to track when the producer has been closed - */ - CompletableFuture closeAsync(); - - /** - * @return Whether the producer is connected to the broker - */ - boolean isConnected(); -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ProducerConfiguration.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ProducerConfiguration.java deleted file mode 100644 index 761c49ec24221..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ProducerConfiguration.java +++ /dev/null @@ -1,474 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import static com.google.common.base.Preconditions.checkArgument; -import java.io.Serializable; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import lombok.EqualsAndHashCode; -import org.apache.pulsar.client.api.PulsarClientException.ProducerBusyException; -import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; - -/** - * Producer's configuration. - * - * @deprecated use {@link PulsarClient#newProducer()} to construct and configure a {@link Producer} instance - */ -@Deprecated -@EqualsAndHashCode -public class ProducerConfiguration implements Serializable { - - private static final long serialVersionUID = 1L; - - private final ProducerConfigurationData conf = new ProducerConfigurationData(); - - @Deprecated - public enum MessageRoutingMode { - SinglePartition, RoundRobinPartition, CustomPartition - } - - @Deprecated - public enum HashingScheme { - JavaStringHash, Murmur3_32Hash - } - - /** - * @return the configured custom producer name or null if no custom name was specified - * @since 1.20.0 - */ - public String getProducerName() { - return conf.getProducerName(); - } - - /** - * Specify a name for the producer - *

- * If not assigned, the system will generate a globally unique name which can be access with - * {@link Producer#getProducerName()}. - *

- * When specifying a name, it is app to the user to ensure that, for a given topic, the producer name is unique - * across all Pulsar's clusters. - *

- * If a producer with the same name is already connected to a particular topic, the - * {@link PulsarClient#createProducer(String)} operation will fail with {@link ProducerBusyException}. - * - * @param producerName - * the custom name to use for the producer - * @since 1.20.0 - */ - public void setProducerName(String producerName) { - conf.setProducerName(producerName); - } - - /** - * @return the message send timeout in ms - */ - public long getSendTimeoutMs() { - return conf.getSendTimeoutMs(); - } - - /** - * Set the send timeout (default: 30 seconds) - *

- * If a message is not acknowledged by the server before the sendTimeout expires, an error will be reported. - * - * @param sendTimeout - * the send timeout - * @param unit - * the time unit of the {@code sendTimeout} - */ - public ProducerConfiguration setSendTimeout(int sendTimeout, TimeUnit unit) { - conf.setSendTimeoutMs(sendTimeout, unit); - return this; - } - - /** - * @return the maximum number of messages allowed in the outstanding messages queue for the producer - */ - public int getMaxPendingMessages() { - return conf.getMaxPendingMessages(); - } - - /** - * Set the max size of the queue holding the messages pending to receive an acknowledgment from the broker. - *

- * When the queue is full, by default, all calls to {@link Producer#send} and {@link Producer#sendAsync} will fail - * unless blockIfQueueFull is set to true. Use {@link #setBlockIfQueueFull} to change the blocking behavior. - * - * @param maxPendingMessages - * @return - */ - public ProducerConfiguration setMaxPendingMessages(int maxPendingMessages) { - conf.setMaxPendingMessages(maxPendingMessages); - return this; - } - - public HashingScheme getHashingScheme() { - return HashingScheme.valueOf(conf.getHashingScheme().toString()); - } - - public ProducerConfiguration setHashingScheme(HashingScheme hashingScheme) { - conf.setHashingScheme(org.apache.pulsar.client.api.HashingScheme.valueOf(hashingScheme.toString())); - return this; - } - - /** - * - * @return the maximum number of pending messages allowed across all the partitions - */ - public int getMaxPendingMessagesAcrossPartitions() { - return conf.getMaxPendingMessagesAcrossPartitions(); - } - - /** - * Set the number of max pending messages across all the partitions - *

- * This setting will be used to lower the max pending messages for each partition - * ({@link #setMaxPendingMessages(int)}), if the total exceeds the configured value. - * - * @param maxPendingMessagesAcrossPartitions - */ - public void setMaxPendingMessagesAcrossPartitions(int maxPendingMessagesAcrossPartitions) { - conf.setMaxPendingMessagesAcrossPartitions(maxPendingMessagesAcrossPartitions); - } - - /** - * - * @return whether the producer will block {@link Producer#send} and {@link Producer#sendAsync} operations when the - * pending queue is full - */ - public boolean getBlockIfQueueFull() { - return conf.isBlockIfQueueFull(); - } - - /** - * Set whether the {@link Producer#send} and {@link Producer#sendAsync} operations should block when the outgoing - * message queue is full. - *

- * Default is false. If set to false, send operations will immediately fail with - * {@link PulsarClientException.ProducerQueueIsFullError} when there is no space left in pending queue. - * - * @param blockIfQueueFull - * whether to block {@link Producer#send} and {@link Producer#sendAsync} operations on queue full - * @return - */ - public ProducerConfiguration setBlockIfQueueFull(boolean blockIfQueueFull) { - conf.setBlockIfQueueFull(blockIfQueueFull); - return this; - } - - /** - * Set the message routing mode for the partitioned producer. - * - * @param messageRouteMode message routing mode. - * @return producer configuration - * @see MessageRoutingMode - */ - public ProducerConfiguration setMessageRoutingMode(MessageRoutingMode messageRouteMode) { - Objects.requireNonNull(messageRouteMode); - conf.setMessageRoutingMode( - org.apache.pulsar.client.api.MessageRoutingMode.valueOf(messageRouteMode.toString())); - return this; - } - - /** - * Get the message routing mode for the partitioned producer. - * - * @return message routing mode, default is round-robin routing. - * @see MessageRoutingMode#RoundRobinPartition - */ - public MessageRoutingMode getMessageRoutingMode() { - return MessageRoutingMode.valueOf(conf.getMessageRoutingMode().toString()); - } - - /** - * Set the compression type for the producer. - *

- * By default, message payloads are not compressed. Supported compression types are: - *

    - *
  • CompressionType.LZ4
  • - *
  • CompressionType.ZLIB
  • - *
- * - * @param compressionType - * @return - * - * @since 1.0.28
- * Make sure all the consumer applications have been updated to use this client version, before starting to - * compress messages. - */ - public ProducerConfiguration setCompressionType(CompressionType compressionType) { - conf.setCompressionType(compressionType); - return this; - } - - /** - * @return the configured compression type for this producer - */ - public CompressionType getCompressionType() { - return conf.getCompressionType(); - } - - /** - * Set a custom message routing policy by passing an implementation of MessageRouter. - * - * - * @param messageRouter - */ - public ProducerConfiguration setMessageRouter(MessageRouter messageRouter) { - Objects.requireNonNull(messageRouter); - setMessageRoutingMode(MessageRoutingMode.CustomPartition); - conf.setCustomMessageRouter(messageRouter); - return this; - } - - /** - * Get the message router set by {@link #setMessageRouter(MessageRouter)}. - * - * @return message router. - * @deprecated since 1.22.0-incubating. numPartitions is already passed as parameter in - * {@link MessageRouter#choosePartition(Message, TopicMetadata)}. - * @see MessageRouter - */ - @Deprecated - public MessageRouter getMessageRouter(int numPartitions) { - return conf.getCustomMessageRouter(); - } - - /** - * Get the message router set by {@link #setMessageRouter(MessageRouter)}. - * - * @return message router set by {@link #setMessageRouter(MessageRouter)}. - */ - public MessageRouter getMessageRouter() { - return conf.getCustomMessageRouter(); - } - - /** - * Return the flag whether automatic message batching is enabled or not. - * - * @return true if batch messages are enabled. otherwise false. - * @since 2.0.0
- * It is enabled by default. - */ - public boolean getBatchingEnabled() { - return conf.isBatchingEnabled(); - } - - /** - * Control whether automatic batching of messages is enabled for the producer. default: false [No batching] - * - * When batching is enabled, multiple calls to Producer.sendAsync can result in a single batch to be sent to the - * broker, leading to better throughput, especially when publishing small messages. If compression is enabled, - * messages will be compressed at the batch level, leading to a much better compression ratio for similar headers or - * contents. - * - * When enabled default batch delay is set to 1 ms and default batch size is 1000 messages - * - * @see ProducerConfiguration#setBatchingMaxPublishDelay(long, TimeUnit) - * @since 1.0.36
- * Make sure all the consumer applications have been updated to use this client version, before starting to - * batch messages. - * - */ - public ProducerConfiguration setBatchingEnabled(boolean batchMessagesEnabled) { - conf.setBatchingEnabled(batchMessagesEnabled); - return this; - } - - /** - * @return the CryptoKeyReader - */ - public CryptoKeyReader getCryptoKeyReader() { - return conf.getCryptoKeyReader(); - } - - /** - * Sets a {@link CryptoKeyReader}. - * - * @param cryptoKeyReader - * CryptoKeyReader object - */ - public ProducerConfiguration setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { - Objects.requireNonNull(cryptoKeyReader); - conf.setCryptoKeyReader(cryptoKeyReader); - return this; - } - - /** - * - * @return encryptionKeys - * - */ - public Set getEncryptionKeys() { - return conf.getEncryptionKeys(); - } - - /** - * - * Returns true if encryption keys are added. - * - */ - public boolean isEncryptionEnabled() { - return conf.isEncryptionEnabled(); - } - - /** - * Add public encryption key, used by producer to encrypt the data key. - * - * At the time of producer creation, Pulsar client checks if there are keys added to encryptionKeys. If keys are - * found, a callback getKey(String keyName) is invoked against each key to load the values of the key. Application - * should implement this callback to return the key in pkcs8 format. If compression is enabled, message is encrypted - * after compression. If batch messaging is enabled, the batched message is encrypted. - * - */ - public void addEncryptionKey(String key) { - conf.getEncryptionKeys().add(key); - } - - public void removeEncryptionKey(String key) { - conf.getEncryptionKeys().remove(key); - } - - /** - * Sets the ProducerCryptoFailureAction to the value specified. - * - * @param action - * The producer action - */ - public void setCryptoFailureAction(ProducerCryptoFailureAction action) { - conf.setCryptoFailureAction(action); - } - - /** - * @return The ProducerCryptoFailureAction - */ - public ProducerCryptoFailureAction getCryptoFailureAction() { - return conf.getCryptoFailureAction(); - } - - /** - * - * @return the batch time period in ms. - * @see ProducerConfiguration#setBatchingMaxPublishDelay(long, TimeUnit) - */ - public long getBatchingMaxPublishDelayMs() { - return TimeUnit.MICROSECONDS.toMillis(conf.getBatchingMaxPublishDelayMicros()); - } - - /** - * Set the time period within which the messages sent will be batched default: 1ms if batch messages are - * enabled. If set to a non zero value, messages will be queued until this time interval or until - * - * @see ProducerConfiguration#batchingMaxMessages threshold is reached; all messages will be published as a single - * batch message. The consumer will be delivered individual messages in the batch in the same order they were - * enqueued - * @since 1.0.36
- * Make sure all the consumer applications have been updated to use this client version, before starting to - * batch messages. - * @param batchDelay - * the batch delay - * @param timeUnit - * the time unit of the {@code batchDelay} - * @return - */ - public ProducerConfiguration setBatchingMaxPublishDelay(long batchDelay, TimeUnit timeUnit) { - conf.setBatchingMaxPublishDelayMicros(batchDelay, timeUnit); - return this; - } - - /** - * - * @return the maximum number of messages permitted in a batch. - */ - public int getBatchingMaxMessages() { - return conf.getBatchingMaxMessages(); - } - - /** - * Set the maximum number of messages permitted in a batch. default: 1000 If set to a value greater than 1, - * messages will be queued until this threshold is reached or batch interval has elapsed - * - * @see ProducerConfiguration#setBatchingMaxPublishDelay(long, TimeUnit) All messages in batch will be published as - * a single batch message. The consumer will be delivered individual messages in the batch in the same order - * they were enqueued - * @param batchMessagesMaxMessagesPerBatch - * maximum number of messages in a batch - * @return - */ - public ProducerConfiguration setBatchingMaxMessages(int batchMessagesMaxMessagesPerBatch) { - conf.setBatchingMaxMessages(batchMessagesMaxMessagesPerBatch); - return this; - } - - public Optional getInitialSequenceId() { - return Optional.ofNullable(conf.getInitialSequenceId()); - } - - /** - * Set the baseline for the sequence ids for messages published by the producer. - *

- * First message will be using (initialSequenceId + 1) as its sequence id and subsequent messages will be assigned - * incremental sequence ids, if not otherwise specified. - * - * @param initialSequenceId - * @return - */ - public ProducerConfiguration setInitialSequenceId(long initialSequenceId) { - conf.setInitialSequenceId(initialSequenceId); - return this; - } - - /** - * Set a name/value property with this producer. - * - * @param key - * @param value - * @return - */ - public ProducerConfiguration setProperty(String key, String value) { - checkArgument(key != null); - checkArgument(value != null); - conf.getProperties().put(key, value); - return this; - } - - /** - * Add all the properties in the provided map. - * - * @param properties - * @return - */ - public ProducerConfiguration setProperties(Map properties) { - conf.getProperties().putAll(properties); - return this; - } - - public Map getProperties() { - return conf.getProperties(); - } - - public ProducerConfigurationData getProducerConfigurationData() { - return conf; - } -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/PulsarClient.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/PulsarClient.java deleted file mode 100644 index 8ac1dfb71e092..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/PulsarClient.java +++ /dev/null @@ -1,273 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import java.io.Closeable; -import java.util.concurrent.CompletableFuture; -import org.apache.pulsar.client.impl.v1.PulsarClientV1Impl; - -/** - * Class that provides a client interface to Pulsar. - *

- * Client instances are thread-safe and can be reused for managing multiple {@link Producer}, {@link Consumer} and - * {@link Reader} instances. - */ -public interface PulsarClient extends Closeable { - - /** - * Create a new PulsarClient object using default client configuration. - * - * @param serviceUrl - * the url of the Pulsar endpoint to be used - * @return a new pulsar client object - * @throws PulsarClientException.InvalidServiceURL - * if the serviceUrl is invalid - * @deprecated use {@link #builder()} to construct a client instance - */ - @Deprecated - static PulsarClient create(String serviceUrl) throws PulsarClientException { - return create(serviceUrl, new ClientConfiguration()); - } - - /** - * Create a new PulsarClient object. - * - * @param serviceUrl - * the url of the Pulsar endpoint to be used - * @param conf - * the client configuration - * @return a new pulsar client object - * @throws PulsarClientException.InvalidServiceURL - * if the serviceUrl is invalid - * @deprecated use {@link #builder()} to construct a client instance - */ - @Deprecated - static PulsarClient create(String serviceUrl, ClientConfiguration conf) throws PulsarClientException { - return new PulsarClientV1Impl(serviceUrl, conf); - } - - /** - * Create a producer with default {@link ProducerConfiguration} for publishing on a specific topic. - * - * @param topic - * The name of the topic where to produce - * @return The producer object - * @throws PulsarClientException.AlreadyClosedException - * if the client was already closed - * @throws PulsarClientException.InvalidTopicNameException - * if the topic name is not valid - * @throws PulsarClientException.AuthenticationException - * if there was an error with the supplied credentials - * @throws PulsarClientException.AuthorizationException - * if the authorization to publish on topic was denied - * @deprecated use {@link #newProducer()} to build a new producer - */ - @Deprecated - Producer createProducer(String topic) throws PulsarClientException; - - /** - * Asynchronously create a producer with default {@link ProducerConfiguration} for publishing on a specific topic. - * - * @param topic - * The name of the topic where to produce - * @return Future of the asynchronously created producer object - * @deprecated use {@link #newProducer()} to build a new producer - */ - @Deprecated - CompletableFuture createProducerAsync(String topic); - - /** - * Create a producer with given {@code ProducerConfiguration} for publishing on a specific topic. - * - * @param topic - * The name of the topic where to produce - * @param conf - * The {@code ProducerConfiguration} object - * @return The producer object - * @throws PulsarClientException - * if it was not possible to create the producer - * @throws InterruptedException - * @deprecated use {@link #newProducer()} to build a new producer - */ - @Deprecated - Producer createProducer(String topic, ProducerConfiguration conf) throws PulsarClientException; - - /** - * Asynchronously create a producer with given {@code ProducerConfiguration} for publishing on a specific topic. - * - * @param topic - * The name of the topic where to produce - * @param conf - * The {@code ProducerConfiguration} object - * @return Future of the asynchronously created producer object - * @deprecated use {@link #newProducer()} to build a new producer - */ - @Deprecated - CompletableFuture createProducerAsync(String topic, ProducerConfiguration conf); - - /** - * Subscribe to the given topic and subscription combination with default {@code ConsumerConfiguration}. - * - * @param topic - * The name of the topic - * @param subscription - * The name of the subscription - * @return The {@code Consumer} object - * @throws PulsarClientException - * @throws InterruptedException - * - * @deprecated Use {@link #newConsumer()} to build a new consumer - */ - @Deprecated - Consumer subscribe(String topic, String subscription) throws PulsarClientException; - - /** - * Asynchronously subscribe to the given topic and subscription combination using default. - * {@code ConsumerConfiguration} - * - * @param topic - * The topic name - * @param subscription - * The subscription name - * @return Future of the {@code Consumer} object - * @deprecated Use {@link #newConsumer()} to build a new consumer - */ - @Deprecated - CompletableFuture subscribeAsync(String topic, String subscription); - - /** - * Subscribe to the given topic and subscription combination with given {@code ConsumerConfiguration}. - * - * @param topic - * The name of the topic - * @param subscription - * The name of the subscription - * @param conf - * The {@code ConsumerConfiguration} object - * @return The {@code Consumer} object - * @throws PulsarClientException - * @deprecated Use {@link #newConsumer()} to build a new consumer - */ - @Deprecated - Consumer subscribe(String topic, String subscription, ConsumerConfiguration conf) throws PulsarClientException; - - /** - * Asynchronously subscribe to the given topic and subscription combination using given. - * {@code ConsumerConfiguration} - * - * @param topic - * The name of the topic - * @param subscription - * The name of the subscription - * @param conf - * The {@code ConsumerConfiguration} object - * @return Future of the {@code Consumer} object - * @deprecated Use {@link #newConsumer()} to build a new consumer - */ - @Deprecated - CompletableFuture subscribeAsync(String topic, String subscription, ConsumerConfiguration conf); - - /** - * Create a topic reader with given {@code ReaderConfiguration} for reading messages from the specified topic. - *

- * The Reader provides a low-level abstraction that allows for manual positioning in the topic, without using a - * subscription. Reader can only work on non-partitioned topics. - *

- * The initial reader positioning is done by specifying a message id. The options are: - *

    - *
  • MessageId.earliest : Start reading from the earliest message available in the topic - *
  • MessageId.latest : Start reading from the end topic, only getting messages published after the - * reader was created - *
  • MessageId : When passing a particular message id, the reader will position itself on that - * specific position. The first message to be read will be the message next to the specified messageId. - *
- * - * @param topic - * The name of the topic where to read - * @param startMessageId - * The message id where the reader will position itself. The first message returned will be the one after - * the specified startMessageId - * @param conf - * The {@code ReaderConfiguration} object - * @return The {@code Reader} object - * @deprecated Use {@link #newReader()} to build a new reader - */ - @Deprecated - Reader createReader(String topic, MessageId startMessageId, ReaderConfiguration conf) throws PulsarClientException; - - /** - * Asynchronously create a topic reader with given {@code ReaderConfiguration} for reading messages from the - * specified topic. - *

- * The Reader provides a low-level abstraction that allows for manual positioning in the topic, without using a - * subscription. Reader can only work on non-partitioned topics. - *

- * The initial reader positioning is done by specifying a message id. The options are: - *

    - *
  • MessageId.earliest : Start reading from the earliest message available in the topic - *
  • MessageId.latest : Start reading from the end topic, only getting messages published after the - * reader was created - *
  • MessageId : When passing a particular message id, the reader will position itself on that - * specific position. The first message to be read will be the message next to the specified messageId. - *
- * - * @param topic - * The name of the topic where to read - * @param startMessageId - * The message id where the reader will position itself. The first message returned will be the one after - * the specified startMessageId - * @param conf - * The {@code ReaderConfiguration} object - * @return Future of the asynchronously created producer object - * @deprecated Use {@link #newReader()} to build a new reader - */ - @Deprecated - CompletableFuture createReaderAsync(String topic, MessageId startMessageId, ReaderConfiguration conf); - - /** - * Close the PulsarClient and release all the resources. - * - * All the producers and consumers will be orderly closed. Waits until all pending write request are persisted. - * - * @throws PulsarClientException - * if the close operation fails - */ - @Override - void close() throws PulsarClientException; - - /** - * Asynchronously close the PulsarClient and release all the resources. - * - * All the producers and consumers will be orderly closed. Waits until all pending write request are persisted. - * - * @throws PulsarClientException - * if the close operation fails - */ - CompletableFuture closeAsync(); - - /** - * Perform immediate shutdown of PulsarClient. - * - * Release all the resources and close all the producers without waiting for ongoing operations to complete. - * - * @throws PulsarClientException - * if the forceful shutdown fails - */ - void shutdown() throws PulsarClientException; -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/Reader.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/Reader.java deleted file mode 100644 index 98fcdb453bb76..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/Reader.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import java.io.Closeable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; - -/** - * A Reader can be used to scan through all the messages currently available in a topic. - * - */ -public interface Reader extends Closeable { - - /** - * @return the topic from which this reader is reading from - */ - String getTopic(); - - /** - * Read the next message in the topic. - * - * @return the next message - * @throws PulsarClientException - */ - Message readNext() throws PulsarClientException; - - /** - * Read the next message in the topic waiting for a maximum of timeout - * time units. Returns null if no message is received in that time. - * - * @return the next message(Could be null if none received in time) - * @throws PulsarClientException - */ - Message readNext(int timeout, TimeUnit unit) throws PulsarClientException; - - CompletableFuture> readNextAsync(); - - /** - * Asynchronously close the reader and stop the broker to push more messages. - * - * @return a future that can be used to track the completion of the operation - */ - CompletableFuture closeAsync(); - - /** - * Return true if the topic was terminated and this reader has reached the end of the topic. - */ - boolean hasReachedEndOfTopic(); - - /** - * Check if there is any message available to read from the current position. - */ - boolean hasMessageAvailable() throws PulsarClientException; - - /** - * Asynchronously Check if there is message that has been published successfully to the broker in the topic. - */ - CompletableFuture hasMessageAvailableAsync(); - - /** - * @return Whether the reader is connected to the broker - */ - boolean isConnected(); -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java deleted file mode 100644 index 885436a11336e..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderConfiguration.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import static com.google.common.base.Preconditions.checkArgument; -import java.io.Serializable; -import java.util.Objects; -import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; -import org.apache.pulsar.client.impl.v1.ReaderV1Impl; - -/** - * - * @deprecated Use {@link PulsarClient#newReader()} to construct and configure a {@link Reader} instance - */ -@Deprecated -public class ReaderConfiguration implements Serializable { - - private final ReaderConfigurationData conf = new ReaderConfigurationData<>(); - - private ReaderListener readerListener; - - /** - * @return the configured {@link ReaderListener} for the reader - */ - public ReaderListener getReaderListener() { - return readerListener; - } - - /** - * Sets a {@link ReaderListener} for the reader - *

- * When a {@link ReaderListener} is set, application will receive messages through it. Calls to - * {@link Reader#readNext()} will not be allowed. - * - * @param readerListener - * the listener object - */ - public ReaderConfiguration setReaderListener(ReaderListener readerListener) { - Objects.requireNonNull(readerListener); - this.readerListener = readerListener; - conf.setReaderListener(new org.apache.pulsar.shade.client.api.v2.ReaderListener() { - - @Override - public void received(org.apache.pulsar.shade.client.api.v2.Reader v2Reader, Message msg) { - readerListener.received(new ReaderV1Impl(v2Reader), msg); - } - - @Override - public void reachedEndOfTopic(org.apache.pulsar.shade.client.api.v2.Reader reader) { - readerListener.reachedEndOfTopic(new ReaderV1Impl(reader)); - } - }); - return this; - } - - /** - * @return the configure receiver queue size value - */ - public int getReceiverQueueSize() { - return conf.getReceiverQueueSize(); - } - - /** - * @return the CryptoKeyReader - */ - public CryptoKeyReader getCryptoKeyReader() { - return conf.getCryptoKeyReader(); - } - - /** - * Sets a {@link CryptoKeyReader}. - * - * @param cryptoKeyReader - * CryptoKeyReader object - */ - public ReaderConfiguration setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { - Objects.requireNonNull(cryptoKeyReader); - conf.setCryptoKeyReader(cryptoKeyReader); - return this; - } - - /** - * Sets the ConsumerCryptoFailureAction to the value specified. - * - * @param action - * The action to take when the decoding fails - */ - public void setCryptoFailureAction(ConsumerCryptoFailureAction action) { - conf.setCryptoFailureAction(action); - } - - /** - * @return The ConsumerCryptoFailureAction - */ - public ConsumerCryptoFailureAction getCryptoFailureAction() { - return conf.getCryptoFailureAction(); - } - - /** - * Sets the size of the consumer receive queue. - *

- * The consumer receive queue controls how many messages can be accumulated by the {@link Consumer} before the - * application calls {@link Consumer#receive()}. Using a higher value could potentially increase the consumer - * throughput at the expense of bigger memory utilization. - *

- * Default value is {@code 1000} messages and should be good for most use cases. - * - * @param receiverQueueSize - * the new receiver queue size value - */ - public ReaderConfiguration setReceiverQueueSize(int receiverQueueSize) { - checkArgument(receiverQueueSize >= 0, "Receiver queue size cannot be negative"); - conf.setReceiverQueueSize(receiverQueueSize); - return this; - } - - /** - * @return the consumer name - */ - public String getReaderName() { - return conf.getReaderName(); - } - - /** - * Set the consumer name. - * - * @param readerName - */ - public ReaderConfiguration setReaderName(String readerName) { - checkArgument(StringUtils.isNotBlank(readerName)); - conf.setReaderName(readerName); - return this; - } - - /** - * @return the subscription role prefix for subscription auth - */ - public String getSubscriptionRolePrefix() { - return conf.getSubscriptionRolePrefix(); - } - - /** - * Set the subscription role prefix for subscription auth. The default prefix is "reader". - * - * @param subscriptionRolePrefix - */ - public ReaderConfiguration setSubscriptionRolePrefix(String subscriptionRolePrefix) { - checkArgument(StringUtils.isNotBlank(subscriptionRolePrefix)); - conf.setSubscriptionRolePrefix(subscriptionRolePrefix); - return this; - } - - public ReaderConfigurationData getReaderConfigurationData() { - return conf; - } - - private static final long serialVersionUID = 1L; -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderListener.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderListener.java deleted file mode 100644 index 26d694d589ac0..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ReaderListener.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.api; - -import java.io.Serializable; - -/** - * A listener that will be called in order for every message received. - */ -public interface ReaderListener extends Serializable { - /** - * This method is called whenever a new message is received. - * - * Messages are guaranteed to be delivered in order and from the same thread for a single consumer - * - * This method will only be called once for each message, unless either application or broker crashes. - * - * Application is responsible of handling any exception that could be thrown while processing the message. - * - * @param reader - * the Reader object from where the message was received - * @param msg - * the message object - */ - void received(Reader reader, Message msg); - - /** - * Get the notification when a topic is terminated. - * - * @param reader - * the Reader object associated with the terminated topic - */ - default void reachedEndOfTopic(Reader reader) { - // By default ignore the notification - } -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/package-info.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/package-info.java deleted file mode 100644 index 57896b5e27235..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/package-info.java +++ /dev/null @@ -1,22 +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. - */ -/** - * Pulsar Client API. - */ -package org.apache.pulsar.client.api; diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/MessageBuilderImpl.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/MessageBuilderImpl.java deleted file mode 100644 index 6d6a08725d7da..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/MessageBuilderImpl.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.impl; - -import static com.google.common.base.Preconditions.checkArgument; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageBuilder; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.common.api.proto.MessageMetadata; - -@SuppressWarnings("deprecation") -public class MessageBuilderImpl implements MessageBuilder { - private static final ByteBuffer EMPTY_CONTENT = ByteBuffer.allocate(0); - private final MessageMetadata msgMetadataBuilder = new MessageMetadata(); - private ByteBuffer content = EMPTY_CONTENT; - - @Override - public Message build() { - return MessageImpl.create(msgMetadataBuilder, content, Schema.BYTES, null); - } - - @Override - public MessageBuilder setContent(byte[] data) { - setContent(data, 0, data.length); - return this; - } - - @Override - public MessageBuilder setContent(byte[] data, int offet, int length) { - this.content = ByteBuffer.wrap(data, offet, length); - return this; - } - - @Override - public MessageBuilder setContent(ByteBuffer buf) { - this.content = buf.duplicate(); - return this; - } - - @Override - public MessageBuilder setProperties(Map properties) { - for (Map.Entry entry : properties.entrySet()) { - msgMetadataBuilder.addProperty() - .setKey(entry.getKey()) - .setValue(entry.getValue()); - } - - return this; - } - - @Override - public MessageBuilder setProperty(String name, String value) { - msgMetadataBuilder.addProperty() - .setKey(name) - .setValue(value); - return this; - } - - @Override - public MessageBuilder setKey(String key) { - msgMetadataBuilder.setPartitionKey(key); - return this; - } - - @Override - public MessageBuilder setEventTime(long timestamp) { - checkArgument(timestamp > 0, "Invalid timestamp : '%s'", timestamp); - msgMetadataBuilder.setEventTime(timestamp); - return this; - } - - @Override - public MessageBuilder setSequenceId(long sequenceId) { - checkArgument(sequenceId >= 0); - msgMetadataBuilder.setSequenceId(sequenceId); - return this; - } - - @Override - public MessageBuilder setReplicationClusters(List clusters) { - Objects.requireNonNull(clusters); - msgMetadataBuilder.clearReplicateTo(); - msgMetadataBuilder.addAllReplicateTos(clusters); - return this; - } - - @Override - public MessageBuilder disableReplication() { - msgMetadataBuilder.clearReplicateTo(); - msgMetadataBuilder.addReplicateTo("__local__"); - return this; - } - - -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/package-info.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/package-info.java deleted file mode 100644 index e429b403ec6b3..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/package-info.java +++ /dev/null @@ -1,22 +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. - */ -/** - * Pulsar Client API. - */ -package org.apache.pulsar.client.impl; diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/ConsumerV1Impl.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/ConsumerV1Impl.java deleted file mode 100644 index ab17beee4f7d2..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/ConsumerV1Impl.java +++ /dev/null @@ -1,176 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.impl.v1; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.ConsumerStats; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; - -public class ConsumerV1Impl implements Consumer { - private final org.apache.pulsar.shade.client.api.v2.Consumer consumer; - - public ConsumerV1Impl(org.apache.pulsar.shade.client.api.v2.Consumer consumer) { - this.consumer = consumer; - } - - @Override - public void acknowledge(Message arg0) throws PulsarClientException { - consumer.acknowledge(arg0); - } - - @Override - public void acknowledge(MessageId arg0) throws PulsarClientException { - consumer.acknowledge(arg0); - } - - @Override - public CompletableFuture acknowledgeAsync(Message arg0) { - return consumer.acknowledgeAsync(arg0); - } - - @Override - public CompletableFuture acknowledgeAsync(MessageId arg0) { - return consumer.acknowledgeAsync(arg0); - } - - @Override - public void acknowledgeCumulative(Message arg0) throws PulsarClientException { - consumer.acknowledgeCumulative(arg0); - } - - @Override - public void acknowledgeCumulative(MessageId arg0) throws PulsarClientException { - consumer.acknowledgeCumulative(arg0); - } - - @Override - public CompletableFuture acknowledgeCumulativeAsync(Message arg0) { - return consumer.acknowledgeCumulativeAsync(arg0); - } - - @Override - public CompletableFuture acknowledgeCumulativeAsync(MessageId arg0) { - return consumer.acknowledgeCumulativeAsync(arg0); - } - - @Override - public void close() throws PulsarClientException { - consumer.close(); - } - - @Override - public CompletableFuture closeAsync() { - return consumer.closeAsync(); - } - - @Override - public String getConsumerName() { - return consumer.getConsumerName(); - } - - @Override - public ConsumerStats getStats() { - return consumer.getStats(); - } - - public String getSubscription() { - return consumer.getSubscription(); - } - - public String getTopic() { - return consumer.getTopic(); - } - - public boolean hasReachedEndOfTopic() { - return consumer.hasReachedEndOfTopic(); - } - - public boolean isConnected() { - return consumer.isConnected(); - } - - public void pause() { - consumer.pause(); - } - - public Message receive() throws PulsarClientException { - return consumer.receive(); - } - - public Message receive(int arg0, TimeUnit arg1) throws PulsarClientException { - return consumer.receive(arg0, arg1); - } - - public CompletableFuture> receiveAsync() { - return consumer.receiveAsync(); - } - - public void redeliverUnacknowledgedMessages() { - consumer.redeliverUnacknowledgedMessages(); - } - - public void resume() { - consumer.resume(); - } - - public void seek(MessageId arg0) throws PulsarClientException { - consumer.seek(arg0); - } - - public void seek(long arg0) throws PulsarClientException { - consumer.seek(arg0); - } - - public void seek(Function function) throws PulsarClientException { - consumer.seek(function); - } - - public CompletableFuture seekAsync(long arg0) { - return consumer.seekAsync(arg0); - } - - public CompletableFuture seekAsync(MessageId arg0) { - return consumer.seekAsync(arg0); - } - - public CompletableFuture seekAsync(Function function) { - return consumer.seekAsync(function); - } - - public void unsubscribe() throws PulsarClientException { - consumer.unsubscribe(); - } - - public CompletableFuture unsubscribeAsync() { - return consumer.unsubscribeAsync(); - } - - public MessageId getLastMessageId() throws PulsarClientException { - return consumer.getLastMessageId(); - } - - public CompletableFuture getLastMessageIdAsync() { - return consumer.getLastMessageIdAsync(); - } -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/ProducerV1Impl.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/ProducerV1Impl.java deleted file mode 100644 index 12c8d0f1527ce..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/ProducerV1Impl.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.impl.v1; - -import java.util.concurrent.CompletableFuture; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerStats; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.impl.ProducerImpl; - -public class ProducerV1Impl implements Producer { - - private final ProducerImpl producer; - - public ProducerV1Impl(ProducerImpl producer) { - this.producer = producer; - } - - public void close() throws PulsarClientException { - producer.close(); - } - - public CompletableFuture closeAsync() { - return producer.closeAsync(); - } - - public void flush() throws PulsarClientException { - producer.flush(); - } - - public CompletableFuture flushAsync() { - return producer.flushAsync(); - } - - public long getLastSequenceId() { - return producer.getLastSequenceId(); - } - - public ProducerStats getStats() { - return producer.getStats(); - } - - public boolean isConnected() { - return producer.isConnected(); - } - - public MessageId send(byte[] value) throws PulsarClientException { - return producer.send(value); - } - - public MessageId send(Message value) throws PulsarClientException { - return producer.send(value); - } - - public CompletableFuture sendAsync(byte[] arg0) { - return producer.sendAsync(arg0); - } - - public CompletableFuture sendAsync(Message arg0) { - return producer.sendAsync(arg0); - } - - @Override - public String getTopic() { - return producer.getTopic(); - } - - @Override - public String getProducerName() { - return producer.getProducerName(); - } -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/PulsarClientV1Impl.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/PulsarClientV1Impl.java deleted file mode 100644 index ca4373d37f4c1..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/PulsarClientV1Impl.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.impl.v1; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.pulsar.client.api.ClientConfiguration; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.ConsumerConfiguration; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerConfiguration; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Reader; -import org.apache.pulsar.client.api.ReaderConfiguration; -import org.apache.pulsar.client.impl.ProducerImpl; -import org.apache.pulsar.client.impl.PulsarClientImpl; -import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; -import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; -import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; -import org.apache.pulsar.common.util.FutureUtil; - -@SuppressWarnings("deprecation") -public class PulsarClientV1Impl implements PulsarClient { - - private final PulsarClientImpl client; - - public PulsarClientV1Impl(String serviceUrl, ClientConfiguration conf) throws PulsarClientException { - this.client = new PulsarClientImpl(conf.setServiceUrl(serviceUrl).getConfigurationData().clone()); - } - - @Override - public void close() throws PulsarClientException { - client.close(); - } - - @Override - public CompletableFuture closeAsync() { - return client.closeAsync(); - } - - @Override - public Producer createProducer(final String topic, final ProducerConfiguration conf) throws PulsarClientException { - if (conf == null) { - throw new PulsarClientException.InvalidConfigurationException("Invalid null configuration object"); - } - - try { - return createProducerAsync(topic, conf).get(); - } catch (ExecutionException e) { - Throwable t = e.getCause(); - if (t instanceof PulsarClientException) { - throw (PulsarClientException) t; - } else { - throw new PulsarClientException(t); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new PulsarClientException(e); - } - } - - @Override - public Producer createProducer(String topic) - throws PulsarClientException { - return createProducer(topic, new ProducerConfiguration()); - } - - @Override - public CompletableFuture createProducerAsync(final String topic, final ProducerConfiguration conf) { - ProducerConfigurationData confData = conf.getProducerConfigurationData().clone(); - confData.setTopicName(topic); - return client.createProducerAsync(confData).thenApply(p -> new ProducerV1Impl((ProducerImpl) p)); - } - - @Override - public CompletableFuture createProducerAsync(String topic) { - return createProducerAsync(topic, new ProducerConfiguration()); - } - - @Override - public Reader createReader(String topic, MessageId startMessageId, ReaderConfiguration conf) - throws PulsarClientException { - try { - return createReaderAsync(topic, startMessageId, conf).get(); - } catch (ExecutionException e) { - Throwable t = e.getCause(); - if (t instanceof PulsarClientException) { - throw (PulsarClientException) t; - } else { - throw new PulsarClientException(t); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new PulsarClientException(e); - } - } - - @Override - public CompletableFuture createReaderAsync(String topic, MessageId startMessageId, - ReaderConfiguration conf) { - ReaderConfigurationData confData = conf.getReaderConfigurationData().clone(); - confData.setTopicName(topic); - confData.setStartMessageId(startMessageId); - return client.createReaderAsync(confData).thenApply(r -> new ReaderV1Impl(r)); - } - - @Override - public void shutdown() throws PulsarClientException { - client.shutdown(); - } - - @Override - public Consumer subscribe(String topic, String subscriptionName) throws PulsarClientException { - return subscribe(topic, subscriptionName, new ConsumerConfiguration()); - } - - @Override - public CompletableFuture subscribeAsync(final String topic, final String subscription, - final ConsumerConfiguration conf) { - if (conf == null) { - return FutureUtil.failedFuture( - new PulsarClientException.InvalidConfigurationException("Invalid null configuration")); - } - - ConsumerConfigurationData confData = conf.getConfigurationData().clone(); - confData.getTopicNames().add(topic); - confData.setSubscriptionName(subscription); - return client.subscribeAsync(confData).thenApply(c -> new ConsumerV1Impl(c)); - } - - @Override - public CompletableFuture subscribeAsync(String topic, - String subscriptionName) { - return subscribeAsync(topic, subscriptionName, new ConsumerConfiguration()); - } - - @Override - public Consumer subscribe(String topic, String subscription, ConsumerConfiguration conf) - throws PulsarClientException { - try { - return subscribeAsync(topic, subscription, conf).get(); - } catch (ExecutionException e) { - Throwable t = e.getCause(); - if (t instanceof PulsarClientException) { - throw (PulsarClientException) t; - } else { - throw new PulsarClientException(t); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new PulsarClientException(e); - } - } -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/ReaderV1Impl.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/ReaderV1Impl.java deleted file mode 100644 index 2e6384459e160..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/ReaderV1Impl.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.impl.v1; - -import java.io.IOException; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Reader; - -public class ReaderV1Impl implements Reader { - - private final org.apache.pulsar.shade.client.api.v2.Reader reader; - - public ReaderV1Impl(org.apache.pulsar.shade.client.api.v2.Reader reader) { - this.reader = reader; - } - - @Override - public void close() throws IOException { - reader.close(); - } - - @Override - public CompletableFuture closeAsync() { - return reader.closeAsync(); - } - - @Override - public String getTopic() { - return reader.getTopic(); - } - - @Override - public boolean hasMessageAvailable() throws PulsarClientException { - return reader.hasMessageAvailable(); - } - - @Override - public CompletableFuture hasMessageAvailableAsync() { - return reader.hasMessageAvailableAsync(); - } - - @Override - public boolean hasReachedEndOfTopic() { - return reader.hasReachedEndOfTopic(); - } - - @Override - public boolean isConnected() { - return reader.isConnected(); - } - - @Override - public Message readNext() throws PulsarClientException { - return reader.readNext(); - } - - @Override - public Message readNext(int arg0, TimeUnit arg1) throws PulsarClientException { - return reader.readNext(arg0, arg1); - } - - @Override - public CompletableFuture> readNextAsync() { - return reader.readNextAsync(); - } -} diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/package-info.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/package-info.java deleted file mode 100644 index 97705e7d567ca..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/impl/v1/package-info.java +++ /dev/null @@ -1,22 +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. - */ -/** - * Pulsar Client API. - */ -package org.apache.pulsar.client.impl.v1; diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/resources/findbugsExclude.xml b/pulsar-client-1x-base/pulsar-client-1x/src/main/resources/findbugsExclude.xml deleted file mode 100644 index 7938e60bf4330..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/resources/findbugsExclude.xml +++ /dev/null @@ -1,48 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml deleted file mode 100644 index 2e316e8e5eee3..0000000000000 --- a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml +++ /dev/null @@ -1,97 +0,0 @@ - - - 4.0.0 - - - org.apache.pulsar - pulsar-client-1x-base - 4.0.0-SNAPSHOT - - - pulsar-client-2x-shaded - Pulsar Client 2.x Shaded API - - - - ${project.groupId} - pulsar-client - ${project.version} - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - ${shadePluginPhase} - - shade - - - true - true - false - - - - org.apache.pulsar:pulsar-client - org.apache.pulsar:pulsar-client-api - - - - - org.apache.pulsar:pulsar-client - - ** - - - - org/bouncycastle/** - - - - - - org.apache.pulsar.client.api - org.apache.pulsar.shade.client.api.v2 - - org.apache.pulsar.client.api.PulsarClient - org.apache.pulsar.client.api.Producer - org.apache.pulsar.client.api.Consumer - org.apache.pulsar.client.api.Reader - org.apache.pulsar.client.api.MessageListener - org.apache.pulsar.client.api.ReaderListener - - - - - - - - - - From d9bc7af60b2e1afc9a1ca4bd8f3505bcd1c4e06b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 14 Oct 2024 12:50:02 +0800 Subject: [PATCH 077/327] [fix] [broker] Topics failed to delete after remove cluster from replicated clusters set and caused OOM (#23360) --- .../SystemTopicBasedTopicPoliciesService.java | 11 ++++- .../NamespaceEventsSystemTopicFactory.java | 8 +++- .../OneWayReplicatorUsingGlobalZKTest.java | 44 +++++++++++++++++++ 3 files changed, 60 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 6ff6408916b1c..cc3938491e637 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -133,7 +133,16 @@ public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName) { if (NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject()) || isSelf(topicName)) { return CompletableFuture.completedFuture(null); } - return sendTopicPolicyEvent(topicName, ActionType.DELETE, null); + TopicName changeEvents = NamespaceEventsSystemTopicFactory.getEventsTopicName(topicName.getNamespaceObject()); + return pulsarService.getNamespaceService().checkTopicExists(changeEvents).thenCompose(topicExistsInfo -> { + // If the system topic named "__change_events" has been deleted, it means all the data in the topic have + // been deleted, so we do not need to delete the message that we want to delete again. + if (!topicExistsInfo.isExists()) { + log.info("Skip delete topic-level policies because {} has been removed before", changeEvents); + return CompletableFuture.completedFuture(null); + } + return sendTopicPolicyEvent(topicName, ActionType.DELETE, null); + }); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java index f5e6c7748d10b..199026bc4c445 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicFactory.java @@ -37,12 +37,16 @@ public NamespaceEventsSystemTopicFactory(PulsarClient client) { } public TopicPoliciesSystemTopicClient createTopicPoliciesSystemTopicClient(NamespaceName namespaceName) { - TopicName topicName = TopicName.get(TopicDomain.persistent.value(), namespaceName, - SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); + TopicName topicName = getEventsTopicName(namespaceName); log.info("Create topic policies system topic client {}", topicName.toString()); return new TopicPoliciesSystemTopicClient(client, topicName); } + public static TopicName getEventsTopicName(NamespaceName namespaceName) { + return TopicName.get(TopicDomain.persistent.value(), namespaceName, + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); + } + public TransactionBufferSnapshotBaseSystemTopicClient createTransactionBufferSystemTopicClient( TopicName systemTopicName, SystemTopicTxnBufferSnapshotService systemTopicTxnBufferSnapshotService, Class schemaType) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java index d99969fbaa7e5..ad877e8f947b7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java @@ -19,16 +19,24 @@ package org.apache.pulsar.broker.service; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.time.Duration; import java.util.Arrays; import java.util.HashSet; +import java.util.Map; +import java.util.Optional; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; @@ -173,4 +181,40 @@ public void testDifferentTopicCreationRule(ReplicationMode replicationMode) thro public void testReplicationCountMetrics() throws Exception { super.testReplicationCountMetrics(); } + + @Test + public void testRemoveCluster() throws Exception { + // Initialize. + final String ns1 = defaultTenant + "/" + "ns_73b1a31afce34671a5ddc48fe5ad7fc8"; + final String topic = "persistent://" + ns1 + "/___tp-5dd50794-7af8-4a34-8a0b-06188052c66a"; + final String topicChangeEvents = "persistent://" + ns1 + "/__change_events"; + admin1.namespaces().createNamespace(ns1); + admin1.namespaces().setNamespaceReplicationClusters(ns1, new HashSet<>(Arrays.asList(cluster1, cluster2))); + admin1.topics().createNonPartitionedTopic(topic); + + // Wait for loading topic up. + Producer p = client1.newProducer(Schema.STRING).topic(topic).create(); + Awaitility.await().untilAsserted(() -> { + Map>> tps = pulsar1.getBrokerService().getTopics(); + assertTrue(tps.containsKey(topic)); + assertTrue(tps.containsKey(topicChangeEvents)); + }); + + // The topics under the namespace of the cluster-1 will be deleted. + // Verify the result. + admin1.namespaces().setNamespaceReplicationClusters(ns1, new HashSet<>(Arrays.asList(cluster2))); + Awaitility.await().atMost(Duration.ofSeconds(120)).untilAsserted(() -> { + Map>> tps = pulsar1.getBrokerService().getTopics(); + assertFalse(tps.containsKey(topic)); + assertFalse(tps.containsKey(topicChangeEvents)); + assertFalse(pulsar1.getNamespaceService().checkTopicExists(TopicName.get(topic)).join().isExists()); + assertFalse(pulsar1.getNamespaceService() + .checkTopicExists(TopicName.get(topicChangeEvents)).join().isExists()); + }); + + // cleanup. + p.close(); + admin2.topics().delete(topic); + admin2.namespaces().deleteNamespace(ns1); + } } From b340a17b8ffbe5022b5f0de18cfc2aeb68f00259 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 14 Oct 2024 09:52:31 +0300 Subject: [PATCH 078/327] [improve][broker][PIP-379] Improve hash collision handling by restoring consumer when other leaves (#23447) --- ...stentHashingStickyKeyConsumerSelector.java | 95 ++++++++++++++++--- .../broker/service/RemovedHashRanges.java | 17 ++++ ...tHashingStickyKeyConsumerSelectorTest.java | 74 ++++++++++++++- .../org/apache/pulsar/client/api/Range.java | 18 ++++ .../apache/pulsar/client/api/RangeTest.java | 32 +++++++ 5 files changed, 218 insertions(+), 18 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java index 2559a02f87df0..06bac02782eff 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import java.util.ArrayList; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -27,6 +28,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import lombok.ToString; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.Range; @@ -40,8 +42,66 @@ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyCons private static final String KEY_SEPARATOR = "\0"; private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); + /** + * Represents a hash ring point entry. + */ + @ToString + private static class HashRingPointEntry { + Consumer selectedConsumer; + private List collidingConsumers; + + /** + * Create a hash ring entry with a selected consumer. + * @param selectedConsumer the selected consumer + */ + HashRingPointEntry(Consumer selectedConsumer) { + this.selectedConsumer = selectedConsumer; + this.collidingConsumers = null; + } + + /** + * Add a colliding consumer to the hash ring entry. Colliding consumers are consumers that have the same hash + * ring point. A colliding consumer is selected when the selected consumer is removed from the hash ring. + * @param consumer the consumer to add + */ + void addCollidingConsumer(Consumer consumer) { + if (collidingConsumers == null) { + collidingConsumers = new LinkedList<>(); + } + collidingConsumers.add(consumer); + } + + /** + * Remove a consumer from the hash ring entry. When the selected consumer is removed, the first colliding + * consumer is selected as the new selected consumer and removed from the colliding consumers list. + * @param consumer the consumer to remove + * @return true if the entry is empty and should be removed from the hash ring + */ + boolean removeConsumer(Consumer consumer) { + if (selectedConsumer == consumer) { + if (collidingConsumers != null) { + selectedConsumer = collidingConsumers.remove(0); + if (collidingConsumers.isEmpty()) { + collidingConsumers = null; + } + } else { + selectedConsumer = null; + } + } else if (collidingConsumers != null) { + // remove using identity comparison + collidingConsumers.removeIf(c -> c == consumer); + if (collidingConsumers.isEmpty()) { + // remove the list instance when there are no more colliding consumers + collidingConsumers = null; + } + } + // return true when the entry is empty and should be removed from the hash ring + return selectedConsumer == null; + } + } + // Consistent-Hash ring - private final NavigableMap hashRing; + private final NavigableMap hashRing; // Tracks the used consumer name indexes for each consumer name private final ConsumerNameIndexTracker consumerNameIndexTracker = new ConsumerNameIndexTracker(); @@ -84,15 +144,16 @@ public CompletableFuture> addConsumer(Consumer int consumerNameIndex = consumerNameIndexTracker.increaseConsumerRefCountAndReturnIndex(consumerIdentityWrapper); int hash = calculateHashForConsumerAndIndex(consumer, consumerNameIndex, i); - // When there's a collision, the entry won't be added to the hash ring. + // When there's a collision, the entry won't be selected in the hash ring. // This isn't a problem with the consumerNameIndexTracker solution since the collisions won't align // for all hash ring points when using the same consumer name. This won't affect the overall // distribution significantly when the number of hash ring points is sufficiently large (>100). - ConsumerIdentityWrapper existing = hashRing.putIfAbsent(hash, consumerIdentityWrapper); + HashRingPointEntry existing = hashRing.putIfAbsent(hash, new HashRingPointEntry(consumer)); if (existing != null) { hashPointCollisions++; - // reduce the ref count which was increased before adding since the consumer was not added - consumerNameIndexTracker.decreaseConsumerRefCount(consumerIdentityWrapper); + // Add the consumer to the colliding consumers list. The first colliding consumer is selected + // when the selected consumer is removed from the hash ring. + existing.addCollidingConsumer(consumer); } else { hashPointsAdded++; } @@ -147,9 +208,15 @@ public Optional removeConsumer(Consumer consumer) { // Remove all the points that were added for this consumer for (int i = 0; i < numberOfPoints; i++) { int hash = calculateHashForConsumerAndIndex(consumer, consumerNameIndex, i); - if (hashRing.remove(hash, consumerIdentityWrapper)) { - consumerNameIndexTracker.decreaseConsumerRefCount(consumerIdentityWrapper); - } + hashRing.compute(hash, (k, hashRingPointEntry) -> { + assert hashRingPointEntry != null : "hash ring entry wasn't found for hash " + hash; + if (hashRingPointEntry.removeConsumer(consumer)) { + // Remove the entry from the hash ring when there are no more consumers + return null; + } + return hashRingPointEntry; + }); + consumerNameIndexTracker.decreaseConsumerRefCount(consumerIdentityWrapper); } } if (!addOrRemoveReturnsImpactedConsumersResult) { @@ -172,12 +239,12 @@ public Consumer select(int hash) { if (hashRing.isEmpty()) { return null; } - Map.Entry ceilingEntry = hashRing.ceilingEntry(hash); + Map.Entry ceilingEntry = hashRing.ceilingEntry(hash); if (ceilingEntry != null) { - return ceilingEntry.getValue().consumer; + return ceilingEntry.getValue().selectedConsumer; } else { // Handle wrap-around in the hash ring, return the first consumer - return hashRing.firstEntry().getValue().consumer; + return hashRing.firstEntry().getValue().selectedConsumer; } } finally { rwLock.readLock().unlock(); @@ -209,8 +276,8 @@ private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapsh int lastKey = -1; Consumer previousConsumer = null; Range previousRange = null; - for (Map.Entry entry: hashRing.entrySet()) { - Consumer consumer = entry.getValue().consumer; + for (Map.Entry entry: hashRing.entrySet()) { + Consumer consumer = entry.getValue().selectedConsumer; Range range; if (consumer == previousConsumer) { // join ranges @@ -226,7 +293,7 @@ private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapsh previousRange = range; } // Handle wrap-around - Consumer firstConsumer = hashRing.firstEntry().getValue().consumer; + Consumer firstConsumer = hashRing.firstEntry().getValue().selectedConsumer; if (lastKey != getKeyHashRange().getEnd()) { Range range; if (firstConsumer == previousConsumer && previousRange.getEnd() == lastKey) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java index 1833c243f8955..d9bd502255b5b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import java.util.Arrays; import java.util.List; import lombok.EqualsAndHashCode; import lombok.ToString; @@ -71,4 +72,20 @@ public boolean containsStickyKey(int stickyKeyHash) { } return false; } + + /** + * Checks if all removed ranges are fully contained in the provided list of ranges. + */ + public boolean isFullyContainedInRanges(List otherRanges) { + return Arrays.stream(sortedRanges).allMatch(range -> + otherRanges.stream().anyMatch(otherRange -> otherRange.contains(range)) + ); + } + + /** + * Returns the removed hash ranges as a list of ranges. + */ + public List asRanges() { + return Arrays.asList(sortedRanges); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java index 6752cd7cfab45..24e14ca3c9c5d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -475,11 +476,11 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() { } @Test - public void testShouldNotContainMappingChangesWhenConsumersLeaveAndRejoinInSameOrder() { + public void testShouldContainMinimalMappingChangesWhenConsumerLeavesAndRejoins() { final ConsistentHashingStickyKeyConsumerSelector selector = - new ConsistentHashingStickyKeyConsumerSelector(200, true); + new ConsistentHashingStickyKeyConsumerSelector(100, true); final String consumerName = "consumer"; - final int numOfInitialConsumers = 200; + final int numOfInitialConsumers = 10; List consumers = new ArrayList<>(); for (int i = 0; i < numOfInitialConsumers; i++) { final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); @@ -498,8 +499,73 @@ public void testShouldNotContainMappingChangesWhenConsumersLeaveAndRejoinInSameO selector.addConsumer(consumers.get(numOfInitialConsumers / 2)); ConsumerHashAssignmentsSnapshot assignmentsAfter = selector.getConsumerHashAssignmentsSnapshot(); + int removedRangesSize = assignmentsBefore.diffRanges(assignmentsAfter).keySet().stream() + .mapToInt(Range::size) + .sum(); + double allowedremovedRangesPercentage = 1; // 1% + int hashRangeSize = selector.getKeyHashRange().size(); + int allowedremovedRanges = (int) (hashRangeSize * (allowedremovedRangesPercentage / 100.0d)); + assertThat(removedRangesSize).describedAs("Allow up to %d%% of total hash range size to be impacted", + allowedremovedRangesPercentage).isLessThan(allowedremovedRanges); + } - assertThat(assignmentsBefore.resolveImpactedConsumers(assignmentsAfter).getRemovedHashRanges()).isEmpty(); + @Test + public void testShouldNotSwapExistingConsumers() { + final ConsistentHashingStickyKeyConsumerSelector selector = + new ConsistentHashingStickyKeyConsumerSelector(200, true); + final String consumerName = "consumer"; + final int consumerCount = 100; + List consumers = new ArrayList<>(); + for (int i = 0; i < consumerCount; i++) { + final Consumer consumer = createMockConsumer(consumerName + i, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + ConsumerHashAssignmentsSnapshot assignmentsBefore = selector.getConsumerHashAssignmentsSnapshot(); + for (int i = 0; i < consumerCount; i++) { + Consumer consumer = consumers.get(i); + + // remove consumer + selector.removeConsumer(consumer); + + ConsumerHashAssignmentsSnapshot assignmentsAfter = selector.getConsumerHashAssignmentsSnapshot(); + assertThat(assignmentsBefore.resolveImpactedConsumers(assignmentsAfter).getRemovedHashRanges()) + .describedAs( + "when a consumer is removed, the removed hash ranges should only be from " + + "the removed consumer") + .containsOnlyKeys(consumer); + assignmentsBefore = assignmentsAfter; + + // add consumer back + selector.addConsumer(consumer); + + assignmentsAfter = selector.getConsumerHashAssignmentsSnapshot(); + List addedConsumerRanges = assignmentsAfter.getRangesByConsumer().get(consumer); + + Map removedHashRanges = + assignmentsBefore.resolveImpactedConsumers(assignmentsAfter).getRemovedHashRanges(); + ConsumerHashAssignmentsSnapshot finalAssignmentsBefore = assignmentsBefore; + assertThat(removedHashRanges).allSatisfy((c, removedHashRange) -> { + assertThat(removedHashRange + .isFullyContainedInRanges(finalAssignmentsBefore.getRangesByConsumer().get(c))) + .isTrue(); + assertThat(removedHashRange + .isFullyContainedInRanges(addedConsumerRanges)) + .isTrue(); + }).describedAs("when a consumer is added back, all removed hash ranges should be ones " + + "that are moved from existing consumers to the new consumer."); + + List allRemovedRanges = + ConsumerHashAssignmentsSnapshot.mergeOverlappingRanges( + removedHashRanges.entrySet().stream().map(Map.Entry::getValue) + .map(RemovedHashRanges::asRanges) + .flatMap(List::stream).collect(Collectors.toCollection(TreeSet::new))); + assertThat(allRemovedRanges) + .describedAs("all removed ranges should be the same as the ranges of the added consumer") + .containsExactlyElementsOf(addedConsumerRanges); + + assignmentsBefore = assignmentsAfter; + } } @Test diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java index cbca1ef8f06bd..3db225330d0c7 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java @@ -94,10 +94,28 @@ public int compareTo(Range o) { return result; } + /** + * Check if the value is in the range. + * @param value + * @return true if the value is in the range. + */ public boolean contains(int value) { return value >= start && value <= end; } + /** + * Check if the range is fully contained in the other range. + * @param otherRange + * @return true if the range is fully contained in the other range. + */ + public boolean contains(Range otherRange) { + return start <= otherRange.start && end >= otherRange.end; + } + + /** + * Get the size of the range. + * @return the size of the range. + */ public int size() { return end - start + 1; } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java index 50168221fea37..bb443498a16bd 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java @@ -87,6 +87,38 @@ public void testContains() { Assert.assertFalse(range.contains(6)); } + @Test + public void testContainsRange() { + Range range = Range.of(5, 10); + + // Test ranges that are fully contained + Assert.assertTrue(range.contains(Range.of(6, 8))); + + Assert.assertTrue(range.contains(Range.of(5, 10))); + + Assert.assertTrue(range.contains(Range.of(5, 5))); + + Assert.assertTrue(range.contains(Range.of(5, 8))); + + Assert.assertTrue(range.contains(Range.of(10, 10))); + + Assert.assertTrue(range.contains(Range.of(8, 10))); + + // Test ranges that are not fully contained + Assert.assertFalse(range.contains(Range.of(1, 5))); + + Assert.assertFalse(range.contains(Range.of(1, 4))); + + Assert.assertFalse(range.contains(Range.of(1, 10))); + + Assert.assertFalse(range.contains(Range.of(1, 11))); + + Assert.assertFalse(range.contains(Range.of(10, 12))); + + Assert.assertFalse(range.contains(Range.of(11, 20))); + } + + @Test public void testSize() { Range range = Range.of(0, 0); From 209fd784765916e28d38e50e52f6ed90d47527fd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 14 Oct 2024 12:03:57 +0300 Subject: [PATCH 079/327] [fix][build] Add basic support for vscode-java and Eclipse IDE (#23448) --- .gitignore | 2 ++ pom.xml | 46 ++++++++++++++++++++++++++ pulsar-broker/pom.xml | 19 +++++++++++ pulsar-common/pom.xml | 32 ++++++++++++++++++ pulsar-transaction/coordinator/pom.xml | 34 +++++++++++++++++-- 5 files changed, 131 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 80d760cd29df7..fe6e44915e628 100644 --- a/.gitignore +++ b/.gitignore @@ -99,3 +99,5 @@ test-reports/ .mvn/.gradle-enterprise/ # Gradle Develocity .mvn/.develocity/ +.vscode +effective-pom.xml diff --git a/pom.xml b/pom.xml index 5734c6fddac92..feefaf97300bc 100644 --- a/pom.xml +++ b/pom.xml @@ -316,6 +316,7 @@ flexible messaging model and an intuitive client API. 0.1.21 1.3 0.4 + 3.6.0 10.0.2 1.2.0 1.6.1 @@ -1927,6 +1928,7 @@ flexible messaging model and an intuitive client API. generated-site/** **/*.md **/.idea/** + **/.vscode/** **/.mvn/** **/generated/** **/zk-3.5-test-data/* @@ -2072,6 +2074,7 @@ flexible messaging model and an intuitive client API. **/SecurityAuth.audit* **/site2/** **/.idea/** + **/.vscode/** **/.mvn/** **/*.a **/*.so @@ -2272,6 +2275,49 @@ flexible messaging model and an intuitive client API. + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.maven.plugins + maven-dependency-plugin + [1.0.0,) + + copy + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + [1.0.0,) + + unpack + + + + + + + + + + diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index a9521e76296de..07d44efa9968a 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -676,6 +676,25 @@ + + org.codehaus.mojo + build-helper-maven-plugin + ${build-helper-maven-plugin.version} + + + add-source + generate-sources + + add-source + + + + target/generated-sources/lightproto/java + + + + + maven-resources-plugin diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 893e350507c08..481185d93e766 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -294,6 +294,38 @@ + + org.codehaus.mojo + build-helper-maven-plugin + ${build-helper-maven-plugin.version} + + + add-source + generate-sources + + add-source + + + + target/generated-sources/protobuf/java + + + + + add-test-source + generate-sources + + add-test-source + + + + target/generated-test-sources/protobuf/java + + + + + + pl.project13.maven diff --git a/pulsar-transaction/coordinator/pom.xml b/pulsar-transaction/coordinator/pom.xml index 9b17487e0ad4d..dbbf74994989f 100644 --- a/pulsar-transaction/coordinator/pom.xml +++ b/pulsar-transaction/coordinator/pom.xml @@ -67,7 +67,7 @@ - + @@ -99,7 +99,37 @@ - + + org.codehaus.mojo + build-helper-maven-plugin + ${build-helper-maven-plugin.version} + + + add-source + generate-sources + + add-source + + + + target/generated-sources/protobuf/java + + + + + add-test-source + generate-sources + + add-test-source + + + + target/generated-test-sources/protobuf/java + + + + + com.github.spotbugs spotbugs-maven-plugin From e2fb0daac8411d76a3cca8df3e8f9f2b5c9b4ed6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 14 Oct 2024 13:19:19 +0300 Subject: [PATCH 080/327] [fix][client] Fix the javadoc for ConsumerBuilder.isAckReceiptEnabled (#23452) --- .../pulsar/client/api/ConsumerBuilder.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index 1b2e5cc5a5e51..4b50f7e0c60b9 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -198,17 +198,21 @@ public interface ConsumerBuilder extends Cloneable { ConsumerBuilder ackTimeout(long ackTimeout, TimeUnit timeUnit); /** - * Acknowledgement returns receipt, but the message is not re-sent after getting receipt. + * Enables or disables the acknowledgment receipt feature. * - * Configure the acknowledgement timeout mechanism to redeliver the message if it is not acknowledged after - * ackTimeout, or to execute a timer task to check the acknowledgement timeout messages during every - * ackTimeoutTickTime period. + *

When this feature is enabled, the consumer ensures that acknowledgments are processed by the broker by + * waiting for a receipt from the broker. Even when the broker returns a receipt, it doesn't guarantee that the + * message won't be redelivered later due to certain implementation details. + * It is recommended to use the asynchronous {@link Consumer#acknowledgeAsync(Message)} method for acknowledgment + * when this feature is enabled. This is because using the synchronous {@link Consumer#acknowledge(Message)} method + * with acknowledgment receipt can cause performance issues due to the round trip to the server, which prevents + * pipelining (having multiple messages in-flight). With the asynchronous method, the consumer can continue + * consuming other messages while waiting for the acknowledgment receipts. * - * @param isAckReceiptEnabled {@link Boolean} enables acknowledgement for receipt + * @param isAckReceiptEnabled {@code true} to enable acknowledgment receipt, {@code false} to disable it * @return the consumer builder instance */ ConsumerBuilder isAckReceiptEnabled(boolean isAckReceiptEnabled); - /** * Define the granularity of the ack-timeout redelivery. * From 0c4108b8208bc84761ca8d03bd111b46ca39bdf8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 14 Oct 2024 14:09:25 +0300 Subject: [PATCH 081/327] [fix][test] Address flaky GetPartitionMetadataMultiBrokerTest (#23456) --- .../pulsar/broker/admin/GetPartitionMetadataTest.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java index e9a639697d9ff..d06f6ea1c56aa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java @@ -47,6 +47,7 @@ import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.tests.TestRetrySupport; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; @@ -56,7 +57,7 @@ @Test(groups = "broker-admin") @Slf4j -public class GetPartitionMetadataTest { +public class GetPartitionMetadataTest extends TestRetrySupport { protected static final String DEFAULT_NS = "public/default"; @@ -72,8 +73,10 @@ public class GetPartitionMetadataTest { protected PulsarClientImpl clientWithHttpLookup1; protected PulsarClientImpl clientWitBinaryLookup1; + @Override @BeforeClass(alwaysRun = true) protected void setup() throws Exception { + incrementSetupNumber(); bkEnsemble = new LocalBookkeeperEnsemble(3, 0, () -> 0); bkEnsemble.start(); // Start broker. @@ -85,8 +88,10 @@ protected void setup() throws Exception { admin1.namespaces().createNamespace(DEFAULT_NS); } + @Override @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { + markCurrentSetupNumberCleaned(); cleanupBrokers(); if (bkEnsemble != null) { bkEnsemble.stop(); @@ -236,7 +241,7 @@ public void testCompatibilityForNewClientAndOldBroker(TopicDomain topicDomain) t String pulsarUrl = pulsar1.getBrokerServiceUrl(); PulsarClientImpl[] clients = getClientsToTest(false); for (PulsarClientImpl client : clients) { - client.getLookup(pulsarUrl).getBroker(TopicName.get(DEFAULT_NS + "/tp1")); + client.getLookup(pulsarUrl).getBroker(TopicName.get(DEFAULT_NS + "/tp1")).join(); } // Inject a not support flag into the connections initialized. Field field = ClientCnx.class.getDeclaredField("supportsGetPartitionedMetadataWithoutAutoCreation"); From 9f7b4877b0349e7f989d510f092883a5109c1c71 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 14 Oct 2024 19:16:45 +0800 Subject: [PATCH 082/327] [improve][pip] PIP-367: Propose a Contributor Repository for Pulsar (#23061) Co-authored-by: xiangying Co-authored-by: xiangying --- pip/pip-367.md | 181 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 181 insertions(+) create mode 100644 pip/pip-367.md diff --git a/pip/pip-367.md b/pip/pip-367.md new file mode 100644 index 0000000000000..8c1eb3e7a26fe --- /dev/null +++ b/pip/pip-367.md @@ -0,0 +1,181 @@ +# PIP-367: Propose a Contributor Repository for Pulsar + +# Background knowledge + +There are now many non-core, controversial features being added to Pulsar. The demand for these features does exist, but +continuously adding too many features to Pulsar can make it more bloated and blur the semantics of some functionalities +provided by Pulsar. + +Here are some examples I encountered while participating in the community. + +Some are questioned for potentially affecting the semantics of Pulsar, but they are ultimately added to Pulsar after +thorough discussions and proper processes; +> Pulsar is being used by many legacy systems and changing its semantics for specific usecases without considering +> consequences are creating a lot of pain and incompatibility problems for other existing systems and let's avoid doing +> it as we are struggling with such changes and +> breaking compatibility or changing semantics are just not acceptable. [0] + +> As for PIP-321 getting in without a consensus, I was one who had concerns with it (and still think poorly of it), but +> I don't think it was decided in violation to the rules. [1] + +Some, after discussion, were abstracted and simplified, leaving the implementation details to the users. [2] + +Additionally, when contributors proposed to contribute more features to the Pulsar Client, they encountered some +skepticism. Although these concerns were addressed after discussions, they also reflected the ongoing dissatisfaction +with the bloated API of Pulsar. +> IMO the surface area the client already exposes is way too big. It feels any person that wanted to scratch their own +> itch just added their feature without holding back. This needs to be maintained. +> In OTel they have a contrib repo where people can contribute their plugin . In your case it's contributing a listener +> implementation. [3] + +The suggestion of the OpenTelemetry contrib repository as a final proposal might be a good solution to this problem. It +can prevent too many new features from being added to Pulsar, which would make Pulsar's semantics vague, giving it a +kind of hodgepodge feeling. At the same time, it can also enrich the ecosystem of Pulsar. +Only vital and critical features will be included in the main repository of Pulsar, while other personalized +requirements should be placed in contributor repositories, similar to the OpenTelemetry contributor repository +approach. [4] + +# Motivation + +Enrich the Pulsar ecosystem by adding a contrib repository for maintaining non-core requirements. + +The main differences between Pulsar's main repository and contributor repositories typically include the following +points: + +**Main Repository** + +- **Core Components**: The main repository houses the core functionalities and critical components of the Pulsar + project, which form the foundation and are essential for all users. +- **Stability**: The code within the main repository is usually subjected to rigorous testing and review to ensure its + stability and reliability. +- **Maintenance**: The main repository is maintained and updated by the project's core maintenance team. +- **Integration**: The code and features from the main repository are typically integrated into Pulsar's official + release versions. +- **Documentation**: The main repository is complemented by comprehensive documentation and examples on the official + website, aiding users in understanding and utilizing the project. + +**Contributor Repository** + +In contrast, contributor repositories might include: + +- **Experimental Features**: These could be in the early stages of development and not yet mature enough to be + incorporated into the main repository. +- **Non-Core Features**: Features that are useful for some users but not necessary for all, thus not fitting into the + main repository. +- **Optimization for Specific Scenarios**: Tailored optimizations for particular use cases or environments, which may + not be universally applicable. +- **Community-Driven**: Typically maintained by community members, these repositories may encompass a variety of + perspectives and implementation approaches. +- **Diversity**: Contributor repositories can offer multiple implementations, allowing users to select based on their + specific needs. +- **Risk**: The use of code from contributor repositories might carry higher risks due to potentially less stringent + testing compared to the main repository. +- **Flexibility**: These repositories allow for faster iteration and experimentation, not being bound by the release + cycle and stability requirements of the main repository. +- **Gradual Integration**: Mature features from contributor repositories are allowed to be gradually integrated into the + main repository. This usually necessitates a series of reviews and tests to ensure they meet the quality standards of + the main repository. +- **Community Participation**: Encourages community members to contribute and expresses gratitude to all those who have + contributed. +- **Documentation**: Focuses on maintaining and showcasing newly introduced features to enhance transparency and + encourage community involvement and contribution. The documentation will exclusively contain information on new + features, including feature descriptions, development progress, and a list of contributors, to acknowledge their + contributions to the project. + +# Goals + +## In Scope + +### Requirement Assessment: + +- Discuss the necessity of establishing a contributor repository and analyze its potential benefits to project + development and community engagement. + +### Project Affiliation: + +- Evaluate and decide whether to make this repository an official subproject of Apache Pulsar for better management and + promotion. + +### Process Design: + +- If the community decides to establish a contributor repository, we will collaboratively design and determine the + contribution process to ensure it is efficient and easy for new contributors to follow. + +### Maintenance Strategy: + +- Develop maintenance processes and strategies, clarifying the responsibilities of maintainers, code review standards, + and quality assurance measures. + +## Out of Scope + +- **Core Feature Decisions**: Deciding which features are core and which are non-core contributions is typically + determined by project maintainers and community leaders based on the project roadmap and vision, and is not within the + direct discussion scope of the contributor repository. + +- **Feature Migration**: Migrating non-core or semantically inconsistent features of Pulsar out of the core repository + involves in-depth analysis and decision-making of the existing codebase, usually handled by the core maintenance team. + +# High Level Design + +Provide a non-core code maintenance repository to collect plugin implementations, personalized features, experimental +features, and user best practices. + +- This repository will not be released +- After iteration, mature features or plugins will be gradually merged into the main branch. + +# Detailed Design + +## Design & Implementation Details + +The contributor repository is different from Pulsar's main contribution process. The contributor repository should take +a more flexible approach to handle the integration of new features: + +1. **Custom interface implementation**: + +- For newly added custom interface implementations, if they do not cause functional conflicts, they can be directly + merged into the main branch after completing the code review. + +2. **Potentially conflicting features**: + +- For new features or optimizations that may conflict with other features, we recommend developing them in a private + repository and maintaining a curated list in the contributor repository. + +3. **Branch review and merge**: + +- Maintainers and community committers review the code and decide whether to allow the plugin to be merged into the main + branch or add experimental features to the curated list. After multiple iterations, widely used and mature features + can be tried to be merged into the Pulsar main repository. + +4. **User-defined integration**: + +- Users can select the required functions according to the description in the function document and integrate them into + their own fork for customized release. + +5. **Version release** + +- Contributor repositories will not be released, users need to release them by themselves. + +For specific repository design, please refer to: https://github.com/StevenLuMT/pulsar-java-contrib/blob/stable/README.md?plain=1 + +# Alternatives + + + +# General Notes + +# Links + +[0] - https://lists.apache.org/thread/87qfp8ht5s0fvw2y4t3j9yzgfmdzmcnz + +[1] - https://lists.apache.org/thread/gzx4j9q0xdtcvrfvvq72t9tm2rt9h3r7 + +[2] - https://github.com/apache/pulsar/pull/22861 + +[3] - https://github.com/apache/pulsar/pull/22902#issuecomment-2168778056 + +[4] - https://github.com/open-telemetry/opentelemetry-java-contrib + +* Mailing List discussion thread: https://lists.apache.org/thread/75y70j6pqw15271b0tq8j637tb02mrwo +* Mailing List voting thread: https://lists.apache.org/thread/td0j8l1c3l93nny0m5smnsdmb91j1n2y From 33475ec456ad4f297d8c77bd33b76db38bc80cef Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Mon, 14 Oct 2024 19:19:05 +0800 Subject: [PATCH 083/327] [improve][broker] Add log to track issue when `handleGetTopicsOfNamespace` (#23434) --- .../main/java/org/apache/pulsar/broker/service/ServerCnx.java | 4 ++-- 1 file changed, 2 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 37b431e833983..c14602bfca507 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 @@ -2510,8 +2510,8 @@ protected void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGet filterTopics = true; filteredTopics = TopicList.filterTopics(filteredTopics, topicsPattern.get()); } else { - log.info("[{}] Subscription pattern provided was longer than maximum {}.", - remoteAddress, maxSubscriptionPatternLength); + log.info("[{}] Subscription pattern provided [{}] was longer than maximum {}.", + remoteAddress, topicsPattern.get(), maxSubscriptionPatternLength); } } String hash = TopicList.calculateHash(filteredTopics); From d759a1e616614ab661c3f858292dcada8f733e63 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 14 Oct 2024 19:54:55 +0300 Subject: [PATCH 084/327] [fix][build] Set project version to 4.1.0-SNAPSHOT (#23442) --- bouncy-castle/bc/pom.xml | 2 +- bouncy-castle/bcfips-include-test/pom.xml | 2 +- bouncy-castle/bcfips/pom.xml | 2 +- bouncy-castle/pom.xml | 2 +- buildtools/pom.xml | 4 ++-- distribution/io/pom.xml | 2 +- distribution/offloaders/pom.xml | 2 +- distribution/pom.xml | 2 +- distribution/server/pom.xml | 2 +- distribution/shell/pom.xml | 2 +- docker/pom.xml | 2 +- docker/pulsar-all/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- jetcd-core-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- microbench/pom.xml | 2 +- pom.xml | 4 ++-- pulsar-bom/pom.xml | 4 ++-- pulsar-broker-auth-athenz/pom.xml | 2 +- pulsar-broker-auth-oidc/pom.xml | 2 +- pulsar-broker-auth-sasl/pom.xml | 2 +- pulsar-broker-common/pom.xml | 2 +- pulsar-broker/pom.xml | 2 +- pulsar-cli-utils/pom.xml | 2 +- pulsar-client-admin-api/pom.xml | 2 +- pulsar-client-admin-shaded/pom.xml | 2 +- pulsar-client-admin/pom.xml | 2 +- pulsar-client-all/pom.xml | 2 +- pulsar-client-api/pom.xml | 2 +- pulsar-client-auth-athenz/pom.xml | 2 +- pulsar-client-auth-sasl/pom.xml | 2 +- pulsar-client-messagecrypto-bc/pom.xml | 2 +- pulsar-client-shaded/pom.xml | 2 +- pulsar-client-tools-api/pom.xml | 2 +- pulsar-client-tools-customcommand-example/pom.xml | 2 +- pulsar-client-tools-test/pom.xml | 2 +- pulsar-client-tools/pom.xml | 2 +- pulsar-client/pom.xml | 2 +- pulsar-common/pom.xml | 2 +- pulsar-config-validation/pom.xml | 2 +- pulsar-docs-tools/pom.xml | 2 +- pulsar-functions/api-java/pom.xml | 2 +- pulsar-functions/instance/pom.xml | 2 +- pulsar-functions/java-examples-builtin/pom.xml | 2 +- pulsar-functions/java-examples/pom.xml | 2 +- pulsar-functions/localrun-shaded/pom.xml | 2 +- pulsar-functions/localrun/pom.xml | 2 +- pulsar-functions/pom.xml | 2 +- pulsar-functions/proto/pom.xml | 2 +- pulsar-functions/runtime-all/pom.xml | 2 +- pulsar-functions/runtime/pom.xml | 2 +- pulsar-functions/secrets/pom.xml | 2 +- pulsar-functions/utils/pom.xml | 2 +- pulsar-functions/worker/pom.xml | 2 +- pulsar-io/aerospike/pom.xml | 2 +- pulsar-io/alluxio/pom.xml | 2 +- pulsar-io/aws/pom.xml | 2 +- pulsar-io/azure-data-explorer/pom.xml | 2 +- pulsar-io/batch-data-generator/pom.xml | 2 +- pulsar-io/batch-discovery-triggerers/pom.xml | 2 +- pulsar-io/canal/pom.xml | 2 +- pulsar-io/cassandra/pom.xml | 2 +- pulsar-io/common/pom.xml | 2 +- pulsar-io/core/pom.xml | 2 +- pulsar-io/data-generator/pom.xml | 2 +- pulsar-io/debezium/core/pom.xml | 2 +- pulsar-io/debezium/mongodb/pom.xml | 2 +- pulsar-io/debezium/mssql/pom.xml | 2 +- pulsar-io/debezium/mysql/pom.xml | 2 +- pulsar-io/debezium/oracle/pom.xml | 2 +- pulsar-io/debezium/pom.xml | 2 +- pulsar-io/debezium/postgres/pom.xml | 2 +- pulsar-io/docs/pom.xml | 2 +- pulsar-io/dynamodb/pom.xml | 2 +- pulsar-io/elastic-search/pom.xml | 2 +- pulsar-io/file/pom.xml | 2 +- pulsar-io/flume/pom.xml | 2 +- pulsar-io/hbase/pom.xml | 2 +- pulsar-io/hdfs3/pom.xml | 2 +- pulsar-io/http/pom.xml | 2 +- pulsar-io/influxdb/pom.xml | 2 +- pulsar-io/jdbc/clickhouse/pom.xml | 2 +- pulsar-io/jdbc/core/pom.xml | 2 +- pulsar-io/jdbc/mariadb/pom.xml | 2 +- pulsar-io/jdbc/openmldb/pom.xml | 2 +- pulsar-io/jdbc/pom.xml | 2 +- pulsar-io/jdbc/postgres/pom.xml | 2 +- pulsar-io/jdbc/sqlite/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor-nar/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor/pom.xml | 2 +- pulsar-io/kafka/pom.xml | 2 +- pulsar-io/kinesis/pom.xml | 2 +- pulsar-io/mongo/pom.xml | 2 +- pulsar-io/netty/pom.xml | 2 +- pulsar-io/nsq/pom.xml | 2 +- pulsar-io/pom.xml | 2 +- pulsar-io/rabbitmq/pom.xml | 2 +- pulsar-io/redis/pom.xml | 2 +- pulsar-io/solr/pom.xml | 2 +- pulsar-io/twitter/pom.xml | 2 +- pulsar-metadata/pom.xml | 2 +- pulsar-opentelemetry/pom.xml | 2 +- pulsar-package-management/bookkeeper-storage/pom.xml | 2 +- pulsar-package-management/core/pom.xml | 2 +- pulsar-package-management/filesystem-storage/pom.xml | 2 +- pulsar-package-management/pom.xml | 2 +- pulsar-proxy/pom.xml | 2 +- pulsar-testclient/pom.xml | 2 +- pulsar-transaction/common/pom.xml | 2 +- pulsar-transaction/coordinator/pom.xml | 2 +- pulsar-transaction/pom.xml | 2 +- pulsar-websocket/pom.xml | 2 +- structured-event-log/pom.xml | 2 +- testmocks/pom.xml | 2 +- tests/bc_2_0_0/pom.xml | 2 +- tests/bc_2_0_1/pom.xml | 2 +- tests/bc_2_6_0/pom.xml | 2 +- tests/docker-images/java-test-functions/pom.xml | 2 +- tests/docker-images/java-test-image/pom.xml | 2 +- tests/docker-images/java-test-plugins/pom.xml | 2 +- tests/docker-images/latest-version-image/pom.xml | 2 +- tests/docker-images/pom.xml | 2 +- tests/integration/pom.xml | 2 +- tests/pom.xml | 2 +- tests/pulsar-client-admin-shade-test/pom.xml | 2 +- tests/pulsar-client-all-shade-test/pom.xml | 2 +- tests/pulsar-client-shade-test/pom.xml | 2 +- tiered-storage/file-system/pom.xml | 2 +- tiered-storage/jcloud/pom.xml | 2 +- tiered-storage/pom.xml | 2 +- 131 files changed, 134 insertions(+), 134 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index e440923af6de5..a8c4a9656db44 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT bouncy-castle-bc diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index f4478174b86dd..441d200327970 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar bouncy-castle-parent - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT bcfips-include-test diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 250b3db6b9b08..2910f15080aa5 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT bouncy-castle-bcfips diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 4d85a163104a2..63c2508c45de2 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT diff --git a/buildtools/pom.xml b/buildtools/pom.xml index b1ae0cd9b73f3..bd467c7285e91 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ org.apache.pulsar buildtools - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT jar Pulsar Build Tools - 2024-08-09T08:42:01Z + 2024-10-14T13:32:50Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 813c4d26d9391..2c9ee5dbd68a6 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT pulsar-io-distribution diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 38beeacde8ba4..1708684cf04f9 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT pulsar-offloader-distribution diff --git a/distribution/pom.xml b/distribution/pom.xml index 67604e145dd73..3523fce8fcec4 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT distribution diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 36641dea20f0c..265d21a672cbe 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT pulsar-server-distribution diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 45108aba68f87..bbe00fc82e041 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT pulsar-shell-distribution diff --git a/docker/pom.xml b/docker/pom.xml index ffcaec3ffdc30..ceeb34a97bea3 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index b43121dd0f613..1f365109c7983 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 481fc319be732..edb6346957d9e 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index dd19faad904bc..b8451cd626d0a 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT jclouds-shaded diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index a0885f8509547..4cafcc1a44b77 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT jetcd-core-shaded diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 22b093f7aafd7..2b83fcb49177b 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT managed-ledger diff --git a/microbench/pom.xml b/microbench/pom.xml index bef02794adbd6..f549cf1408ec0 100644 --- a/microbench/pom.xml +++ b/microbench/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT microbench diff --git a/pom.xml b/pom.xml index feefaf97300bc..3cba432491ab7 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ org.apache.pulsar pulsar - 4.0.0-SNAPSHOT + 4.1.0-SNAPSHOT Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -94,7 +94,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-08-09T08:42:01Z + 2024-10-14T13:32:50Z true - diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index f71f61df228f1..d20a2ef208e27 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -71,8 +71,16 @@ org.apache.avro avro + + org.bouncycastle + bcprov-jdk15on + + + org.bouncycastle + bcprov-jdk18on + jakarta.activation diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index 4cbee957f764d..e44b08cbd96d3 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -53,9 +53,16 @@ dnsjava dnsjava + + org.bouncycastle + bcprov-jdk15on + - + + org.bouncycastle + bcprov-jdk18on + org.apache.hadoop hadoop-hdfs-client @@ -85,6 +92,10 @@ dnsjava dnsjava + + org.bouncycastle + bcprov-jdk15on + @@ -133,6 +144,10 @@ dnsjava dnsjava + + org.bouncycastle + bcprov-jdk15on + From 6bd03088bd366c036390c2b99b865d952866d8bf Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 1 Nov 2024 03:47:16 -0700 Subject: [PATCH 111/327] [fix][broker] Fix Broker migration NPE while broker tls url not configured (#23534) --- .../pulsar/broker/service/AbstractTopic.java | 4 ++-- .../broker/service/ClusterMigrationTest.java | 2 +- .../org/apache/pulsar/client/impl/ClientCnx.java | 9 ++++++--- .../apache/pulsar/common/protocol/Commands.java | 14 +++++++++----- 4 files changed, 18 insertions(+), 11 deletions(-) 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 11f00fb28e34b..96ea2004be8d7 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 @@ -1363,8 +1363,8 @@ public static CompletableFuture> getMigratedClusterUrlAsync .getClusterPoliciesAsync(pulsar.getConfig().getClusterName()) .thenCombine(isNamespaceMigrationEnabledAsync(pulsar, topic), ((clusterData, isNamespaceMigrationEnabled) -> { - Optional url = ((clusterData.isPresent() && clusterData.get().isMigrated()) - || isNamespaceMigrationEnabled) + Optional url = (clusterData.isPresent() && (clusterData.get().isMigrated() + || isNamespaceMigrationEnabled)) ? Optional.ofNullable(clusterData.get().getMigratedClusterUrl()) : Optional.empty(); return url; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java index e56a3495600f0..e6a7d049366e4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java @@ -297,7 +297,7 @@ public void testClusterMigration() throws Exception { assertFalse(topic2.getProducers().isEmpty()); ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getWebServiceAddress(), pulsar2.getWebServiceAddressTls(), - pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); + pulsar2.getBrokerServiceUrl(), null); admin1.clusters().updateClusterMigration("r1", true, migratedUrl); assertEquals(admin1.clusters().getClusterMigration("r1").getMigratedClusterUrl(), migratedUrl); 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 24163c631ffe9..35c41455e8987 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 @@ -710,9 +710,12 @@ protected void handleReachedEndOfTopic(CommandReachedEndOfTopic commandReachedEn @Override protected void handleTopicMigrated(CommandTopicMigrated commandTopicMigrated) { final long resourceId = commandTopicMigrated.getResourceId(); - final String serviceUrl = commandTopicMigrated.getBrokerServiceUrl(); - final String serviceUrlTls = commandTopicMigrated.getBrokerServiceUrlTls(); - + final String serviceUrl = commandTopicMigrated.hasBrokerServiceUrl() + ? commandTopicMigrated.getBrokerServiceUrl() + : null; + final String serviceUrlTls = commandTopicMigrated.hasBrokerServiceUrlTls() + ? commandTopicMigrated.getBrokerServiceUrlTls() + : null; HandlerState resource = commandTopicMigrated.getResourceType() == ResourceType.Producer ? producers.get(resourceId) : consumers.get(resourceId); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 8635368f00f0b..19aa9907549d9 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -88,6 +88,7 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.CommandTcClientConnectResponse; +import org.apache.pulsar.common.api.proto.CommandTopicMigrated; import org.apache.pulsar.common.api.proto.CommandTopicMigrated.ResourceType; import org.apache.pulsar.common.api.proto.FeatureFlags; import org.apache.pulsar.common.api.proto.IntRange; @@ -768,11 +769,14 @@ public static ByteBuf newReachedEndOfTopic(long consumerId) { public static ByteBuf newTopicMigrated(ResourceType type, long resourceId, String brokerUrl, String brokerUrlTls) { BaseCommand cmd = localCmd(Type.TOPIC_MIGRATED); - cmd.setTopicMigrated() - .setResourceType(type) - .setResourceId(resourceId) - .setBrokerServiceUrl(brokerUrl) - .setBrokerServiceUrlTls(brokerUrlTls); + CommandTopicMigrated migratedCmd = cmd.setTopicMigrated(); + migratedCmd.setResourceType(type).setResourceId(resourceId); + if (StringUtils.isNotBlank(brokerUrl)) { + migratedCmd.setBrokerServiceUrl(brokerUrl); + } + if (StringUtils.isNotBlank(brokerUrlTls)) { + migratedCmd.setBrokerServiceUrlTls(brokerUrlTls); + } return serializeWithSize(cmd); } From 76d0e6c9171aaeba456736e44574494f3ae3dede Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 1 Nov 2024 13:16:30 -0700 Subject: [PATCH 112/327] [fix][broker] Fix print cluster migration state response (#23535) --- .../src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java index 9f11b48513867..7ea66ef36f5ad 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java @@ -163,7 +163,7 @@ private class GetClusterMigration extends CliCommand { @Override void run() throws PulsarAdminException { - getAdmin().clusters().getClusterMigration(cluster); + print(getAdmin().clusters().getClusterMigration(cluster)); } } From 570cb443eb220135861abf9e9de34ae65e5cdaaa Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 2 Nov 2024 11:12:55 +0200 Subject: [PATCH 113/327] [improve][test] Disable OTel autoconfigured exporters in tests (#23540) --- .../mledger/impl/cache/InflightReadsLimiterTest.java | 4 ++++ .../pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java | 8 ++++++++ .../pulsar/broker/testcontext/PulsarTestContext.java | 3 ++- .../pulsar/opentelemetry/OpenTelemetryServiceTest.java | 5 +++++ 4 files changed, 19 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java index 281f19e388837..68135598e3339 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java @@ -28,6 +28,7 @@ import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; +import java.util.Map; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; @@ -207,6 +208,9 @@ private Pair buildOpenTelemetryAndReader var metricReader = InMemoryMetricReader.create(); var openTelemetry = AutoConfiguredOpenTelemetrySdk.builder() .disableShutdownHook() + .addPropertiesSupplier(() -> Map.of("otel.metrics.exporter", "none", + "otel.traces.exporter", "none", + "otel.logs.exporter", "none")) .addMeterProviderCustomizer((builder, __) -> builder.registerMetricReader(metricReader)) .build() .getOpenTelemetrySdk(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java index 69653ea26e240..0d46e80a70302 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java @@ -36,12 +36,20 @@ public static Consumer getOpenTelemetrySd sdkBuilder.addMeterProviderCustomizer( (meterProviderBuilder, __) -> meterProviderBuilder.registerMetricReader(reader)); sdkBuilder.disableShutdownHook(); + disableExporters(sdkBuilder); sdkBuilder.addPropertiesSupplier( () -> Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false", "otel.java.enabled.resource.providers", "none")); }; } + public static void disableExporters(AutoConfiguredOpenTelemetrySdkBuilder sdkBuilder) { + sdkBuilder.addPropertiesSupplier(() -> + Map.of("otel.metrics.exporter", "none", + "otel.traces.exporter", "none", + "otel.logs.exporter", "none")); + } + public static void assertMetricDoubleSumValue(Collection metrics, String metricName, Attributes attributes, Consumer valueConsumer) { assertThat(metrics) 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 cdb047079bfcd..6403c3bcec4c3 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 @@ -761,7 +761,8 @@ protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { var reader = InMemoryMetricReader.create(); openTelemetryMetricReader(reader); registerCloseable(reader); - openTelemetrySdkBuilderCustomizer = BrokerOpenTelemetryTestUtil.getOpenTelemetrySdkBuilderConsumer(reader); + openTelemetrySdkBuilderCustomizer = + BrokerOpenTelemetryTestUtil.getOpenTelemetrySdkBuilderConsumer(reader); } else { openTelemetrySdkBuilderCustomizer = null; } diff --git a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java index f1a7ffaa12289..99d4189d8f803 100644 --- a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java +++ b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java @@ -76,6 +76,11 @@ private static Consumer getBuilderCustomi (sdkMeterProviderBuilder, __) -> sdkMeterProviderBuilder.registerMetricReader(extraReader)); } autoConfigurationCustomizer.disableShutdownHook(); + // disable all autoconfigured exporters + autoConfigurationCustomizer.addPropertiesSupplier(() -> + Map.of("otel.metrics.exporter", "none", + "otel.traces.exporter", "none", + "otel.logs.exporter", "none")); autoConfigurationCustomizer.addPropertiesSupplier(() -> extraProperties); }; } From 7a4788895e31dcd794fcb89b3af2bc36fa221343 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Tue, 5 Nov 2024 08:19:22 +0800 Subject: [PATCH 114/327] [fix][broker] Increase readBuffer size for bookkeeper.DLOutputStream (#23548) --- .../storage/bookkeeper/DLOutputStream.java | 17 +++++++++++++---- .../storage/bookkeeper/DLOutputStreamTest.java | 2 +- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/pulsar-package-management/bookkeeper-storage/src/main/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStream.java b/pulsar-package-management/bookkeeper-storage/src/main/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStream.java index 67345ebd47e31..f446961c1d8fe 100644 --- a/pulsar-package-management/bookkeeper-storage/src/main/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStream.java +++ b/pulsar-package-management/bookkeeper-storage/src/main/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStream.java @@ -36,7 +36,16 @@ class DLOutputStream { private final DistributedLogManager distributedLogManager; private final AsyncLogWriter writer; - private final byte[] readBuffer = new byte[8192]; + /* + * The LogRecord structure is: + * ------------------- + * Bytes 0 - 7 : Metadata (Long) + * Bytes 8 - 15 : TxId (Long) + * Bytes 16 - 19 : Payload length (Integer) + * Bytes 20 - 20+payload.length-1 : Payload (Byte[]) + * So the max buffer size should be LogRecord.MAX_LOGRECORD_SIZE - 2 * (Long.SIZE / 8) - Integer.SIZE / 8 + */ + private final byte[] readBuffer = new byte[LogRecord.MAX_LOGRECORD_SIZE - 2 * (Long.SIZE / 8) - Integer.SIZE / 8]; private long offset = 0L; private DLOutputStream(DistributedLogManager distributedLogManager, AsyncLogWriter writer) { @@ -51,9 +60,9 @@ static CompletableFuture openWriterAsync(DistributedLogManager d private void writeAsyncHelper(InputStream is, CompletableFuture result) { try { - int read = is.read(readBuffer); - if (read != -1) { - log.info("write something into the ledgers offset: {}, length: {}", offset, read); + int read = is.readNBytes(readBuffer, 0, readBuffer.length); + if (read > 0) { + log.debug("write something into the ledgers offset: {}, length: {}", offset, read); final ByteBuf writeBuf = Unpooled.wrappedBuffer(readBuffer, 0, read); offset += writeBuf.readableBytes(); final LogRecord record = new LogRecord(offset, writeBuf); diff --git a/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStreamTest.java b/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStreamTest.java index b55e0e0d34a4f..235cb4fefc0c3 100644 --- a/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStreamTest.java +++ b/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStreamTest.java @@ -99,7 +99,7 @@ public void writeBytesArrayData() throws ExecutionException, InterruptedExceptio @Test public void writeLongBytesArrayData() throws ExecutionException, InterruptedException { - byte[] data = new byte[8192 * 3 + 4096]; + byte[] data = new byte[1040364 * 3 + 4096]; DLOutputStream.openWriterAsync(dlm) .thenCompose(w -> w.writeAsync(new ByteArrayInputStream(data)) .thenCompose(DLOutputStream::closeAsync)).get(); From 3f12269d8d99e514cf48ef6d57fc3928d37b3646 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 5 Nov 2024 11:40:39 +0800 Subject: [PATCH 115/327] [improve][io] Support update subscription position for sink connector (#23538) --- .../pulsar/functions/utils/SinkConfigUtils.java | 4 +++- .../functions/utils/SinkConfigUtilsTest.java | 14 ++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SinkConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SinkConfigUtils.java index 6631c053fac49..65b6b97fc6ee9 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SinkConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SinkConfigUtils.java @@ -724,7 +724,9 @@ public static SinkConfig validateUpdate(SinkConfig existingConfig, SinkConfig ne if (newConfig.getTransformFunctionConfig() != null) { mergedConfig.setTransformFunctionConfig(newConfig.getTransformFunctionConfig()); } - + if (newConfig.getSourceSubscriptionPosition() != null) { + mergedConfig.setSourceSubscriptionPosition(newConfig.getSourceSubscriptionPosition()); + } return mergedConfig; } diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/SinkConfigUtilsTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/SinkConfigUtilsTest.java index 5c2b6d92b9366..c4c79a635eac0 100644 --- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/SinkConfigUtilsTest.java +++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/SinkConfigUtilsTest.java @@ -38,6 +38,7 @@ import lombok.Data; import lombok.NoArgsConstructor; import lombok.experimental.Accessors; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.common.functions.ConsumerConfig; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.Resources; @@ -224,6 +225,18 @@ public void testCleanSubscriptionField() throws IOException { } } + @Test + public void testUpdateSubscriptionPosition() { + SinkConfig sinkConfig = createSinkConfig(); + SinkConfig newSinkConfig = createSinkConfig(); + newSinkConfig.setSourceSubscriptionPosition(SubscriptionInitialPosition.Earliest); + SinkConfig mergedConfig = SinkConfigUtils.validateUpdate(sinkConfig, newSinkConfig); + assertEquals( + new Gson().toJson(newSinkConfig), + new Gson().toJson(mergedConfig) + ); + } + @Test public void testMergeEqual() { SinkConfig sinkConfig = createSinkConfig(); @@ -565,6 +578,7 @@ private SinkConfig createSinkConfig() { inputSpecs.put("test-input", ConsumerConfig.builder().isRegexPattern(true).serdeClassName("test-serde").build()); sinkConfig.setInputSpecs(inputSpecs); sinkConfig.setProcessingGuarantees(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE); + sinkConfig.setSourceSubscriptionPosition(SubscriptionInitialPosition.Latest); sinkConfig.setRetainOrdering(false); sinkConfig.setRetainKeyOrdering(false); sinkConfig.setConfigs(new HashMap<>()); From c53a85ec909d0b760fbcdd945bee6b6929922b63 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 5 Nov 2024 16:36:27 +0800 Subject: [PATCH 116/327] [fix] [admin] Fix lookup get a null result if uses proxy (#23556) --- .../org/apache/pulsar/client/admin/internal/LookupImpl.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java index ff7a02ca3275e..2482f2cc7a521 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java @@ -24,6 +24,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import javax.ws.rs.client.WebTarget; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.admin.Lookup; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Topics; @@ -57,7 +58,8 @@ public CompletableFuture lookupTopicAsync(String topic) { WebTarget path = v2lookup.path(prefix).path(topicName.getLookupName()); return asyncGetRequest(path, new FutureCallback() {}) - .thenApply(lookupData -> useTls ? lookupData.getBrokerUrlTls() : lookupData.getBrokerUrl()); + .thenApply(lookupData -> useTls && StringUtils.isNotBlank(lookupData.getBrokerUrlTls()) + ? lookupData.getBrokerUrlTls() : lookupData.getBrokerUrl()); } @Override From d2d05c2fc5dd0d89be63fea1fdcd927726ff85e5 Mon Sep 17 00:00:00 2001 From: CrazyCoder <18235787078@163.com> Date: Tue, 5 Nov 2024 18:16:55 +0800 Subject: [PATCH 117/327] [fix][test] Fix ManagedCursorTest.testForceCursorRecovery (#23518) --- .../org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 587f87a7d1d38..b4ab673facb26 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -4828,7 +4828,6 @@ public void operationFailed(ManagedLedgerException exception) { @Test void testForceCursorRecovery() throws Exception { - ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig(); TestPulsarMockBookKeeper bk = new TestPulsarMockBookKeeper(executor); factory = new ManagedLedgerFactoryImpl(metadataStore, bk); ManagedLedgerConfig config = new ManagedLedgerConfig(); @@ -4874,8 +4873,9 @@ public void asyncOpenLedger(final long lId, final DigestType digestType, final b final OpenCallback cb, final Object ctx) { if (ledgerErrors.containsKey(lId)) { cb.openComplete(ledgerErrors.get(lId), null, ctx); + } else { + super.asyncOpenLedger(lId, digestType, passwd, cb, ctx); } - super.asyncOpenLedger(lId, digestType, passwd, cb, ctx); } } From ce0c1bbd9d328dd697548f3e7cde5de10e5aff72 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 5 Nov 2024 12:40:15 -0800 Subject: [PATCH 118/327] [fix][broker] fix logging with correct error message while loading the topic (#23544) --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 fee5e25647ce6..77cd52f4558ea 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 @@ -1875,7 +1875,10 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { }, () -> isTopicNsOwnedByBrokerAsync(topicName), null); }).exceptionally((exception) -> { - log.warn("[{}] Failed to get topic configuration: {}", topic, exception.getMessage(), exception); + boolean migrationFailure = exception.getCause() instanceof TopicMigratedException; + String msg = migrationFailure ? "Topic is already migrated" : + "Failed to get topic configuration:"; + log.warn("[{}] {} {}", topic, msg, exception.getMessage(), exception); // remove topic from topics-map in different thread to avoid possible deadlock if // createPersistentTopic-thread only tries to handle this future-result pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); From c15a0d698231ec697f3ec45f7f09eb904d56758c Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 6 Nov 2024 09:37:21 +0800 Subject: [PATCH 119/327] [improve][admin] Print error log if handle http response fails (#23563) --- .../client/admin/internal/http/AsyncHttpConnector.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java index de694534a9e25..b04c19b7ded45 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java @@ -272,7 +272,11 @@ public String getReasonPhrase() { if (response.hasResponseBody()) { jerseyResponse.setEntityStream(response.getResponseBodyAsStream()); } - callback.response(jerseyResponse); + try { + callback.response(jerseyResponse); + } catch (Exception ex) { + log.error("failed to handle the http response {}", jerseyResponse, ex); + } } })); return responseFuture; From c4ee362f7e97b91549b7b13224315fe758b6e1b6 Mon Sep 17 00:00:00 2001 From: CrazyCoder <18235787078@163.com> Date: Wed, 6 Nov 2024 11:24:08 +0800 Subject: [PATCH 120/327] [fix][test] Fix SimpleProducerConsumerTest.testMultiTopicsConsumerImplPauseForManualSubscription (#23546) --- .../apache/pulsar/client/api/SimpleProducerConsumerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index a867dc0cd3562..e99abac1ec40f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -3520,7 +3520,7 @@ public void testMultiTopicsConsumerImplPauseForManualSubscription() throws Excep producer3.send(message.getBytes(UTF_8)); } - int receiverQueueSize = 1; + int receiverQueueSize = 6; Consumer consumer = pulsarClient .newConsumer() .topics(Lists.newArrayList(topicNameBase + "1", topicNameBase + "2")) From 8eeb0e2e89f8938348d29044d9e1d843a6251067 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 6 Nov 2024 17:01:33 +0800 Subject: [PATCH 121/327] [fix][broker] Fix currently client retries until operation timeout if the topic does not exist (#23530) --- .../pulsar/broker/service/ServerCnx.java | 2 +- .../admin/GetPartitionMetadataTest.java | 3 +- ...okenAuthenticatedProducerConsumerTest.java | 71 ++++++++++++++++--- .../pulsar/client/impl/PulsarClientImpl.java | 4 +- 4 files changed, 67 insertions(+), 13 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 c14602bfca507..f9e593345d85f 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 @@ -713,7 +713,7 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa Throwable actEx = FutureUtil.unwrapCompletionException(ex); if (actEx instanceof WebApplicationException restException) { if (restException.getResponse().getStatus() == Response.Status.NOT_FOUND.getStatusCode()) { - writeAndFlush(Commands.newPartitionMetadataResponse(ServerError.MetadataError, + writeAndFlush(Commands.newPartitionMetadataResponse(ServerError.TopicNotFound, "Tenant or namespace or topic does not exist: " + topicName.getNamespace() , requestId)); lookupSemaphore.release(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java index d06f6ea1c56aa..f3237676c8630 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java @@ -599,8 +599,7 @@ public void testNamespaceNotExist(TopicDomain topicDomain) throws Exception { fail("Expected a not found ex"); } catch (Exception ex) { Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); - assertTrue(unwrapEx instanceof PulsarClientException.BrokerMetadataException || - unwrapEx instanceof PulsarClientException.TopicDoesNotExistException); + assertTrue(unwrapEx instanceof PulsarClientException.TopicDoesNotExistException); } } // Verify: lookup semaphore has been releases. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java index f8ae0279e08b7..9da3fcbd0edc9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java @@ -35,6 +35,7 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.impl.auth.AuthenticationToken; @@ -42,8 +43,9 @@ import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -56,31 +58,35 @@ public class TokenAuthenticatedProducerConsumerTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(TokenAuthenticatedProducerConsumerTest.class); + private final static String ADMIN_ROLE = "admin"; private final String ADMIN_TOKEN; + private final String USER_TOKEN; private final String TOKEN_PUBLIC_KEY; + private final KeyPair kp; TokenAuthenticatedProducerConsumerTest() throws NoSuchAlgorithmException { KeyPairGenerator kpg = KeyPairGenerator.getInstance("RSA"); - KeyPair kp = kpg.generateKeyPair(); + kp = kpg.generateKeyPair(); byte[] encodedPublicKey = kp.getPublic().getEncoded(); TOKEN_PUBLIC_KEY = "data:;base64," + Base64.getEncoder().encodeToString(encodedPublicKey); - ADMIN_TOKEN = generateToken(kp); + ADMIN_TOKEN = generateToken(ADMIN_ROLE); + USER_TOKEN = generateToken("user"); } - private String generateToken(KeyPair kp) { + private String generateToken(String subject) { PrivateKey pkey = kp.getPrivate(); long expMillis = System.currentTimeMillis() + Duration.ofHours(1).toMillis(); Date exp = new Date(expMillis); return Jwts.builder() - .setSubject("admin") + .setSubject(subject) .setExpiration(exp) .signWith(pkey, SignatureAlgorithm.forSigningKey(pkey)) .compact(); } - @BeforeMethod + @BeforeClass @Override protected void setup() throws Exception { conf.setAuthenticationEnabled(true); @@ -118,7 +124,7 @@ protected final void clientSetup() throws Exception { .authentication(AuthenticationFactory.token(ADMIN_TOKEN))); } - @AfterMethod(alwaysRun = true) + @AfterClass(alwaysRun = true) @Override protected void cleanup() throws Exception { super.internalCleanup(); @@ -172,4 +178,53 @@ public void testTokenProducerAndConsumer() throws Exception { log.info("-- Exiting {} test --", methodName); } + @DataProvider + public static Object[][] provider() { + // The 1st element specifies whether to use TCP service URL + // The 2nd element specifies whether to use a token with correct permission + return new Object[][] { + { true, true }, + { true, false }, + { false, true }, + { false, false }, + }; + } + + @Test(dataProvider = "provider") + public void testTopicNotFound(boolean useTcpServiceUrl, boolean useCorrectToken) throws Exception { + final var operationTimeoutMs = 10000; + final var url = useTcpServiceUrl ? pulsar.getBrokerServiceUrl() : pulsar.getWebServiceAddress(); + final var token = useCorrectToken ? ADMIN_TOKEN : USER_TOKEN; + @Cleanup final var client = PulsarClient.builder().serviceUrl(url) + .operationTimeout(operationTimeoutMs, TimeUnit.MILLISECONDS) + .authentication(AuthenticationFactory.token(token)).build(); + final var topic = "my-property/not-exist/tp"; // the namespace does not exist + var start = System.currentTimeMillis(); + try { + client.newProducer().topic(topic).create(); + Assert.fail(); + } catch (PulsarClientException e) { + final var elapsedMs = System.currentTimeMillis() - start; + log.info("Failed to create producer after {} ms: {} {}", elapsedMs, e.getClass().getName(), e.getMessage()); + Assert.assertTrue(elapsedMs < operationTimeoutMs); + if (useTcpServiceUrl) { + Assert.assertTrue(e instanceof PulsarClientException.TopicDoesNotExistException); + } else { + Assert.assertTrue(e instanceof PulsarClientException.NotFoundException); + } + } + start = System.currentTimeMillis(); + try { + client.newConsumer().topic(topic).subscriptionName("sub").subscribe(); + } catch (PulsarClientException e) { + final var elapsedMs = System.currentTimeMillis() - start; + log.info("Failed to subscribe after {} ms: {} {}", elapsedMs, e.getClass().getName(), e.getMessage()); + Assert.assertTrue(elapsedMs < operationTimeoutMs); + if (useTcpServiceUrl) { + Assert.assertTrue(e instanceof PulsarClientException.TopicDoesNotExistException); + } else { + Assert.assertTrue(e instanceof PulsarClientException.NotFoundException); + } + } + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 603844eeb786e..871666620b7b4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -418,9 +418,9 @@ private CompletableFuture checkPartitions(String topic, boolean forceNo } }).exceptionally(ex -> { Throwable actEx = FutureUtil.unwrapCompletionException(ex); - if (forceNoPartitioned && actEx instanceof PulsarClientException.NotFoundException + if (forceNoPartitioned && (actEx instanceof PulsarClientException.NotFoundException || actEx instanceof PulsarClientException.TopicDoesNotExistException - || actEx instanceof PulsarAdminException.NotFoundException) { + || actEx instanceof PulsarAdminException.NotFoundException)) { checkPartitions.complete(0); } else { checkPartitions.completeExceptionally(ex); From 13133c11834ec782aa90187abcc8793e87925008 Mon Sep 17 00:00:00 2001 From: VisionXu Date: Wed, 6 Nov 2024 19:41:32 +0800 Subject: [PATCH 122/327] [fix][test]Flaky-test: SchemaServiceTest.testSchemaRegistryMetrics (#23566) --- .../broker/service/schema/SchemaServiceTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java index 658ea268c644c..7e8aa72338024 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java @@ -180,22 +180,22 @@ public void testSchemaRegistryMetrics() throws Exception { Assert.assertEquals(putMetrics.size(), 0); Collection deleteLatency = metrics.get("pulsar_schema_del_ops_latency_count"); - for (Metric metric : deleteLatency) { + assertThat(deleteLatency).anySatisfy(metric -> { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); - } + }); Collection getLatency = metrics.get("pulsar_schema_get_ops_latency_count"); - for (Metric metric : getLatency) { + assertThat(getLatency).anySatisfy(metric -> { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); - } + }); Collection putLatency = metrics.get("pulsar_schema_put_ops_latency_count"); - for (Metric metric : putLatency) { + assertThat(putLatency).anySatisfy(metric -> { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); - } + }); } @Test From c266db236df5f57bf59df1b25a23521419b2ad02 Mon Sep 17 00:00:00 2001 From: CrazyCoder <18235787078@163.com> Date: Wed, 6 Nov 2024 20:56:57 +0800 Subject: [PATCH 123/327] [fix][test] Fix DeadLetterTopicTest.testDeadLetterTopicWithInitialSubscriptionAndMultiConsumers (#23552) --- .../org/apache/pulsar/client/api/DeadLetterTopicTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java index f5a74dcd1661b..7d34ce33e0a7f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java @@ -1010,7 +1010,7 @@ public void testDeadLetterTopicWithInitialSubscriptionAndMultiConsumers() throws .maxRedeliverCount(maxRedeliveryCount) .initialSubscriptionName(dlqInitialSub) .build()) - .receiverQueueSize(100) + .receiverQueueSize(20) .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); @@ -1023,7 +1023,7 @@ public void testDeadLetterTopicWithInitialSubscriptionAndMultiConsumers() throws .maxRedeliverCount(maxRedeliveryCount) .initialSubscriptionName(dlqInitialSub) .build()) - .receiverQueueSize(100) + .receiverQueueSize(20) .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); From c811801de1e6d850d7edbdcc607b91924c9c91f6 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 6 Nov 2024 07:12:16 -0800 Subject: [PATCH 124/327] [fix][client] Add logging when lib disabled sni even after passing proxy-url (#23545) Co-authored-by: Lari Hotari --- .../java/org/apache/pulsar/client/impl/ConnectionPool.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java index 0f49b77b05765..9be734bee6173 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java @@ -127,8 +127,12 @@ public ConnectionPool(InstrumentProvider instrumentProvider, this.eventLoopGroup = eventLoopGroup; this.clientConfig = conf; this.maxConnectionsPerHosts = conf.getConnectionsPerBroker(); - this.isSniProxy = clientConfig.isUseTls() && clientConfig.getProxyProtocol() != null + boolean sniProxyExpected = clientConfig.getProxyProtocol() != null && StringUtils.isNotBlank(clientConfig.getProxyServiceUrl()); + this.isSniProxy = clientConfig.isUseTls() && sniProxyExpected; + if (!this.isSniProxy && sniProxyExpected) { + log.warn("Disabling SNI proxy because tls is not enabled"); + } pool = new ConcurrentHashMap<>(); bootstrap = new Bootstrap(); From 576d34144c2dd44ed2eb0ce0b2babdf95ade874b Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 7 Nov 2024 11:49:28 +0800 Subject: [PATCH 125/327] [fix][broker] Fix ownership loss (#23515) Signed-off-by: Zixuan Liu --- .../pulsar/broker/namespace/OwnershipCache.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java index 9a4534f538774..868ed2d9fc2c1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java @@ -170,7 +170,22 @@ public CompletableFuture> getOwnerAsync(Namespa // If we're not the owner, we need to check if anybody else is String path = ServiceUnitUtils.path(suName); - return lockManager.readLock(path); + return lockManager.readLock(path).thenCompose(owner -> { + // If the current broker is the owner, attempt to reacquire ownership to avoid cache loss. + if (owner.isPresent() && owner.get().equals(selfOwnerInfo)) { + log.warn("Detected ownership loss for broker [{}] on namespace bundle [{}]. " + + "Attempting to reacquire ownership to maintain cache consistency.", + selfOwnerInfo, suName); + try { + return tryAcquiringOwnership(suName).thenApply(Optional::ofNullable); + } catch (Exception e) { + log.error("Failed to reacquire ownership for namespace bundle [{}] on broker [{}]: {}", + suName, selfOwnerInfo, e.getMessage(), e); + return CompletableFuture.failedFuture(e); + } + } + return CompletableFuture.completedFuture(owner); + }); } /** From b5484f6ea9d6ae74d63eff270b8a85fa2b012246 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Fri, 8 Nov 2024 09:00:23 +0800 Subject: [PATCH 126/327] [improve][pip] PIP-380: Support setting up specific namespaces to skipping the load-shedding (#23304) --- pip/pip_380.md | 80 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 80 insertions(+) create mode 100644 pip/pip_380.md diff --git a/pip/pip_380.md b/pip/pip_380.md new file mode 100644 index 0000000000000..f03b255eb2416 --- /dev/null +++ b/pip/pip_380.md @@ -0,0 +1,80 @@ +# PIP-380: Support setting up specific namespaces to skipping the load-shedding + +# Background knowledge + +Some of the Pulsar protocols, for example the KoP will use a system topic to store the metadata, eg: the topic's offset, +when the bundle owner changes the topic's data will need to be full recovered to the broker, and when unloaded, +it needs to remove the broker local memory, if the topic's data were huge, it may cause slow loading. + +# Motivation + +Support setting up specific namespaces to skipping the load-shedding, for the system topics namespace, +or for benchmark-relevant namespaces we better be skipping the load shedding because for system topics, +if the ownership transfers to another broker, it may need to recover the data by replaying the messages, +it will cause message pub/sub temporary unavailable. + +# Goals + +## In Scope + +- Support setting up specific namespaces to skipping the load-shedding +- Use `RoundRobinBrokerSelector` to assign the skipped namespaces bundle. + +# High Level Design + +Add new configuration `loadBalancerSheddingExcludedNamespaces` to the `ServiceConfiguration`. +While doing the load-shedding, filter out the configuration namespaces. + +For shedding excluded namespaces, use `RoundRobinBrokerSelector` to assign the ownership, it can make the assignment +more average because these will not automatically rebalance to another broker unless manually unloaded it. + +# Detailed Design + +## Design & Implementation Details + +## Public-facing Changes + +### Configuration + +Add new configuration `loadBalancerSheddingExcludedNamespaces` to the `ServiceConfiguration`. + +```java + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The namespaces skip for load shedding" + ) + private Set loadBalancerSheddingExcludedNamespaces = new TreeSet<>(); +``` + +# Monitoring + +No new metrics are added in this proposal. + +# Security Considerations + +No new security considerations are added in this proposal. + +# Backward & Forward Compatibility + +## Revert + +No changes are needed to revert to the previous version. + +## Upgrade + +No other changes are needed to upgrade to the new version. + +# Alternatives + +None + +# General Notes + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/316nmco393tzv30gwqdz53rwpj7mzfx8 +* Mailing List voting thread: https://lists.apache.org/thread/yd645km49z5x6t2c0215moncymb1r19n From 137df29f85798b00de75460a1acb91c7bc25453f Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Fri, 8 Nov 2024 10:59:06 +0800 Subject: [PATCH 127/327] [fix][client] fix the beforeConsume() method earlier hit with message listener (#23578) --- .../pulsar/client/api/InterceptorsTest.java | 78 +++++++++++++++++++ .../pulsar/client/impl/ConsumerBase.java | 1 + .../pulsar/client/impl/ConsumerImpl.java | 3 +- .../client/impl/MultiTopicsConsumerImpl.java | 2 +- 4 files changed, 82 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java index 8115f34121d3c..f71cdc551411b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java @@ -476,6 +476,84 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId consumer.close(); } + @Test(dataProvider = "topicPartition") + public void testDoNotEarlierHitBeforeConsumerWithMessageListener(int partitions) throws Exception { + + AtomicInteger beforeConsumeCount = new AtomicInteger(0); + PulsarClient client = PulsarClient.builder() + .serviceUrl(lookupUrl.toString()) + .listenerThreads(1) + .build(); + + ConsumerInterceptor interceptor = new ConsumerInterceptor<>() { + @Override + public void close() { + } + + @Override + public Message beforeConsume(Consumer consumer, Message message) { + beforeConsumeCount.incrementAndGet(); + log.info("beforeConsume messageId: {}", message.getMessageId()); + return message; + } + + @Override + public void onAcknowledge(Consumer consumer, MessageId messageId, Throwable cause) { + } + + @Override + public void onAcknowledgeCumulative(Consumer consumer, MessageId messageId, Throwable cause) { + } + + @Override + public void onNegativeAcksSend(Consumer consumer, Set messageIds) { + } + + @Override + public void onAckTimeoutSend(Consumer consumer, Set messageIds) { + } + }; + + final String topicName = "persistent://my-property/my-ns/my-topic"; + + if (partitions > 0) { + admin.topics().createPartitionedTopic(topicName, partitions); + } else { + admin.topics().createNonPartitionedTopic(topicName); + } + + Consumer consumer = client.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionType(SubscriptionType.Shared) + .intercept(interceptor) + .subscriptionName("my-subscription") + .messageListener((c, m) -> { + // Simulate a long processing time + try { + Thread.sleep(60000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }) + .subscribe(); + + Producer producer = client.newProducer(Schema.STRING) + .topic("persistent://my-property/my-ns/my-topic") + .create(); + + final int messages = 10; + for (int i = 0; i < messages; i++) { + producer.newMessage().value("Hello Pulsar!").send(); + } + Awaitility.await().untilAsserted(() -> { + // Ensure that the interceptor is not hit before the message listener + Assert.assertEquals(beforeConsumeCount.get(), 1); + }); + producer.close(); + consumer.close(); + client.close(); + } + @Test public void testConsumerInterceptorWithPatternTopicSubscribe() throws PulsarClientException { 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 3073f3a833487..0fc906b6e7a9e 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 @@ -1180,6 +1180,7 @@ protected void callMessageListener(Message msg) { id = msg.getMessageId(); } unAckedMessageTracker.add(id, msg.getRedeliveryCount()); + beforeConsume(msg); listener.received(ConsumerBase.this, msg); } catch (Throwable t) { log.error("[{}][{}] Message listener error in processing message: {}", topic, subscription, 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 be01bd00eb300..004adab56f529 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 @@ -542,7 +542,8 @@ protected Message internalReceive(long timeout, TimeUnit unit) throws PulsarC return null; } messageProcessed(message); - return beforeConsume(message); + message = listener == null ? beforeConsume(message) : message; + return message; } catch (InterruptedException e) { ExceptionHandler.handleInterruptedException(e); State state = getState(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index ff293af230838..528a140b81c2c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -401,7 +401,7 @@ protected Message internalReceive(long timeout, TimeUnit unit) throws PulsarC decreaseIncomingMessageSize(message); checkArgument(message instanceof TopicMessageImpl); trackUnAckedMsgIfNoListener(message.getMessageId(), message.getRedeliveryCount()); - message = beforeConsume(message); + message = listener == null ? beforeConsume(message) : message; } resumeReceivingFromPausedConsumersIfNeeded(); return message; From 7822dca1ffe45324d7af7ef830c617f6881a5431 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Mon, 11 Nov 2024 12:26:06 -0800 Subject: [PATCH 128/327] [fix][broker] Broker is failing to create non-durable sub if topic is fenced (#23579) --- .../mledger/impl/ManagedLedgerImpl.java | 3 ++- .../service/persistent/PersistentTopic.java | 5 ++++ .../api/SimpleProducerConsumerTest.java | 27 +++++++++++++++++++ 3 files changed, 34 insertions(+), 1 deletion(-) 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 c1081761b601f..926e7c7f8108e 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 @@ -3930,7 +3930,8 @@ private void checkManagedLedgerIsOpen() throws ManagedLedgerException { } } - synchronized void setFenced() { + @VisibleForTesting + public synchronized void setFenced() { log.info("{} Moving to Fenced state", name); STATE_UPDATER.set(this, State.Fenced); } 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 541c8a7a225e8..651d12373628b 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 @@ -1050,6 +1050,11 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St } else if (ex.getCause() instanceof BrokerServiceException.SubscriptionFencedException && isCompactionSubscription(subscriptionName)) { log.warn("[{}] Failed to create compaction subscription: {}", topic, ex.getMessage()); + } else if (ex.getCause() instanceof ManagedLedgerFencedException) { + // If the topic has been fenced, we cannot continue using it. We need to close and reopen + log.warn("[{}][{}] has been fenced. closing the topic {}", topic, subscriptionName, + ex.getMessage()); + close(); } else { log.error("[{}] Failed to create subscription: {}", topic, subscriptionName, ex); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index e99abac1ec40f..78d28e4b22834 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -4979,4 +4979,31 @@ private int compareMessageIds(MessageIdImpl messageId1, MessageIdImpl messageId2 return 0; } } + + @Test + public void testFencedLedger() throws Exception { + log.info("-- Starting {} test --", methodName); + + final String topic = "persistent://my-property/my-ns/fencedLedger"; + + @Cleanup + PulsarClient newPulsarClient = PulsarClient.builder().serviceUrl(lookupUrl.toString()).build(); + + @Cleanup + Producer producer = newPulsarClient.newProducer().topic(topic).enableBatching(false).create(); + + final int numMessages = 5; + for (int i = 0; i < numMessages; i++) { + producer.newMessage().value(("value-" + i).getBytes(UTF_8)).eventTime((i + 1) * 100L).sendAsync(); + } + producer.flush(); + + PersistentTopic pTopic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); + ManagedLedgerImpl ml = (ManagedLedgerImpl) pTopic.getManagedLedger(); + ml.setFenced(); + + Reader reader = newPulsarClient.newReader().topic(topic).startMessageId(MessageId.earliest) + .createAsync().get(5, TimeUnit.SECONDS); + assertNotNull(reader); + } } From d7433d0f3d383651bd0c49b1858ebf7da50dbe1c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 12 Nov 2024 03:09:14 +0000 Subject: [PATCH 129/327] [fix][sec] Upgrade Zookeeper to 3.9.3 to address CVE-2024-51504 (#23581) --- distribution/server/src/assemble/LICENSE.bin.txt | 12 ++++++------ pom.xml | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 790315de42a81..9847b5fec438d 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -464,9 +464,9 @@ The Apache Software License, Version 2.0 - org.apache.avro-avro-1.11.4.jar - org.apache.avro-avro-protobuf-1.11.4.jar * Apache Curator - - org.apache.curator-curator-client-5.1.0.jar - - org.apache.curator-curator-framework-5.1.0.jar - - org.apache.curator-curator-recipes-5.1.0.jar + - org.apache.curator-curator-client-5.7.1.jar + - org.apache.curator-curator-framework-5.7.1.jar + - org.apache.curator-curator-recipes-5.7.1.jar * Apache Yetus - org.apache.yetus-audience-annotations-0.12.0.jar * Kubernetes Client @@ -498,9 +498,9 @@ The Apache Software License, Version 2.0 - io.vertx-vertx-web-4.5.10.jar - io.vertx-vertx-web-common-4.5.10.jar * Apache ZooKeeper - - org.apache.zookeeper-zookeeper-3.9.2.jar - - org.apache.zookeeper-zookeeper-jute-3.9.2.jar - - org.apache.zookeeper-zookeeper-prometheus-metrics-3.9.2.jar + - org.apache.zookeeper-zookeeper-3.9.3.jar + - org.apache.zookeeper-zookeeper-jute-3.9.3.jar + - org.apache.zookeeper-zookeeper-prometheus-metrics-3.9.3.jar * Snappy Java - org.xerial.snappy-snappy-java-1.1.10.5.jar * Google HTTP Client diff --git a/pom.xml b/pom.xml index bf47646e16c71..ebdf3492d3f3b 100644 --- a/pom.xml +++ b/pom.xml @@ -140,12 +140,12 @@ flexible messaging model and an intuitive client API. 1.26.0 4.17.1 - 3.9.2 + 3.9.3 1.5.0 1.10.0 1.1.10.5 4.1.12.1 - 5.1.0 + 5.7.1 4.1.113.Final 0.0.24.Final 9.4.56.v20240826 From 096986933c2627e9067ee6085ab692431db22883 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Tue, 12 Nov 2024 16:59:47 +0800 Subject: [PATCH 130/327] [improve][broker] Support cleanup `replication cluster` and `allowed cluster` when cluster metadata teardown (#23561) --- .../pulsar/PulsarClusterMetadataTeardown.java | 43 ++++++++++- .../ClusterMetadataTeardownTest.java | 74 ++++++++++++++++++- 2 files changed, 112 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataTeardown.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataTeardown.java index 964a49fe10f20..30a0dabea9812 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataTeardown.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataTeardown.java @@ -19,8 +19,10 @@ package org.apache.pulsar; import com.google.protobuf.InvalidProtocolBufferException; +import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import lombok.Cleanup; import org.apache.bookkeeper.client.BKException; @@ -29,12 +31,18 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; +import org.apache.pulsar.broker.resources.NamespaceResources; +import org.apache.pulsar.broker.resources.PulsarResources; +import org.apache.pulsar.broker.resources.TenantResources; import org.apache.pulsar.broker.service.schema.SchemaStorageFormat.SchemaLocator; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.docs.tools.CmdGenerateDocs; 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.MetadataStoreFactory; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -153,12 +161,45 @@ public static void main(String[] args) throws Exception { MetadataStoreConfig.builder().sessionTimeoutMillis(arguments.zkSessionTimeoutMillis) .configFilePath(arguments.configurationStoreConfigPath) .metadataStoreName(MetadataStoreConfig.CONFIGURATION_METADATA_STORE).build()); - deleteRecursively(configMetadataStore, "/admin/clusters/" + arguments.cluster).join(); + PulsarResources resources = new PulsarResources(metadataStore, configMetadataStore); + // Cleanup replication cluster from all tenants and namespaces + TenantResources tenantResources = resources.getTenantResources(); + NamespaceResources namespaceResources = resources.getNamespaceResources(); + List tenants = tenantResources.listTenants(); + for (String tenant : tenants) { + List namespaces = namespaceResources.listNamespacesAsync(tenant).get(); + for (String namespace : namespaces) { + namespaceResources.setPolicies(NamespaceName.get(tenant, namespace), policies -> { + policies.replication_clusters.remove(arguments.cluster); + return policies; + }); + } + removeCurrentClusterFromAllowedClusters(tenantResources, tenant, arguments.cluster); + } + try { + resources.getClusterResources().deleteCluster(arguments.cluster); + } catch (MetadataStoreException.NotFoundException ex) { + // Ignore if the cluster does not exist + log.info("Cluster metadata for '{}' does not exist.", arguments.cluster); + } } log.info("Cluster metadata for '{}' teardown.", arguments.cluster); } + private static void removeCurrentClusterFromAllowedClusters( + TenantResources tenantResources, String tenant, String curCluster) + throws MetadataStoreException, InterruptedException, ExecutionException { + Optional tenantInfoOptional = tenantResources.getTenant(tenant); + if (tenantInfoOptional.isEmpty()) { + return; + } + tenantResources.updateTenantAsync(tenant, ti -> { + ti.getAllowedClusters().remove(curCluster); + return ti; + }).get(); + } + private static CompletableFuture deleteRecursively(MetadataStore metadataStore, String path) { return metadataStore.getChildren(path) .thenCompose(children -> FutureUtil.waitForAll( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataTeardownTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataTeardownTest.java index 5184afade9c85..c689bb60fedf7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataTeardownTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataTeardownTest.java @@ -21,6 +21,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; import java.util.SortedMap; import org.apache.pulsar.PulsarClusterMetadataSetup; import org.apache.pulsar.PulsarClusterMetadataTeardown; @@ -54,7 +55,7 @@ void cleanup() { @Test public void testSetupClusterMetadataAndTeardown() throws Exception { String[] args1 = { - "--cluster", "testReSetupClusterMetadata-cluster", + "--cluster", "cluster1", "--zookeeper", "127.0.0.1:" + localZkS.getZookeeperPort(), "--configuration-store", "127.0.0.1:" + localZkS.getZookeeperPort(), "--configuration-metadata-store-config-path", "src/test/resources/conf/zk_client_enable_sasl.conf", @@ -65,7 +66,7 @@ public void testSetupClusterMetadataAndTeardown() throws Exception { }; PulsarClusterMetadataSetup.main(args1); SortedMap data1 = localZkS.dumpData(); - String clusterDataJson = data1.get("/admin/clusters/testReSetupClusterMetadata-cluster"); + String clusterDataJson = data1.get("/admin/clusters/cluster1"); assertNotNull(clusterDataJson); ClusterData clusterData = ObjectMapperFactory .getMapper() @@ -78,13 +79,78 @@ public void testSetupClusterMetadataAndTeardown() throws Exception { assertFalse(clusterData.isBrokerClientTlsEnabled()); String[] args2 = { - "--cluster", "testReSetupClusterMetadata-cluster", + "--cluster", "cluster1", "--zookeeper", "127.0.0.1:" + localZkS.getZookeeperPort(), "--configuration-store", "127.0.0.1:" + localZkS.getZookeeperPort(), "--configuration-metadata-store-config-path", "src/test/resources/conf/zk_client_enable_sasl.conf", }; PulsarClusterMetadataTeardown.main(args2); SortedMap data2 = localZkS.dumpData(); - assertFalse(data2.containsKey("/admin/clusters/testReSetupClusterMetadata-cluster")); + assertFalse(data2.containsKey("/admin/clusters/cluster1")); + } + + @Test + public void testSetupMultipleClusterMetadataAndTeardown() throws Exception { + String[] cluster1Args = { + "--cluster", "cluster1", + "--zookeeper", "127.0.0.1:" + localZkS.getZookeeperPort(), + "--configuration-store", "127.0.0.1:" + localZkS.getZookeeperPort(), + "--configuration-metadata-store-config-path", "src/test/resources/conf/zk_client_enable_sasl.conf", + "--web-service-url", "http://127.0.0.1:8080", + "--web-service-url-tls", "https://127.0.0.1:8443", + "--broker-service-url", "pulsar://127.0.0.1:6650", + "--broker-service-url-tls", "pulsar+ssl://127.0.0.1:6651" + }; + PulsarClusterMetadataSetup.main(cluster1Args); + String[] cluster2Args = { + "--cluster", "cluster2", + "--zookeeper", "127.0.0.1:" + localZkS.getZookeeperPort(), + "--configuration-store", "127.0.0.1:" + localZkS.getZookeeperPort(), + "--configuration-metadata-store-config-path", "src/test/resources/conf/zk_client_enable_sasl.conf", + "--web-service-url", "http://127.0.0.1:8081", + "--web-service-url-tls", "https://127.0.0.1:8445", + "--broker-service-url", "pulsar://127.0.0.1:6651", + "--broker-service-url-tls", "pulsar+ssl://127.0.0.1:6652" + }; + PulsarClusterMetadataSetup.main(cluster2Args); + SortedMap data1 = localZkS.dumpData(); + String clusterDataJson = data1.get("/admin/clusters/cluster1"); + assertNotNull(clusterDataJson); + ClusterData clusterData = ObjectMapperFactory + .getMapper() + .reader() + .readValue(clusterDataJson, ClusterData.class); + assertEquals(clusterData.getServiceUrl(), "http://127.0.0.1:8080"); + assertEquals(clusterData.getServiceUrlTls(), "https://127.0.0.1:8443"); + assertEquals(clusterData.getBrokerServiceUrl(), "pulsar://127.0.0.1:6650"); + assertEquals(clusterData.getBrokerServiceUrlTls(), "pulsar+ssl://127.0.0.1:6651"); + assertFalse(clusterData.isBrokerClientTlsEnabled()); + + String[] args2 = { + "--cluster", "cluster1", + "--zookeeper", "127.0.0.1:" + localZkS.getZookeeperPort(), + "--configuration-store", "127.0.0.1:" + localZkS.getZookeeperPort(), + "--configuration-metadata-store-config-path", "src/test/resources/conf/zk_client_enable_sasl.conf", + }; + PulsarClusterMetadataTeardown.main(args2); + SortedMap data2 = localZkS.dumpData(); + assertFalse(data2.containsKey("/admin/clusters/cluster1")); + assertTrue(data2.containsKey("/admin/clusters/cluster2")); + + assertTrue(data2.containsKey("/admin/policies/public")); + assertFalse(data2.get("/admin/policies/public").contains("cluster1")); + assertTrue(data2.get("/admin/policies/public").contains("cluster2")); + + assertTrue(data2.containsKey("/admin/policies/pulsar")); + assertFalse(data2.get("/admin/policies/pulsar").contains("cluster1")); + assertTrue(data2.get("/admin/policies/pulsar").contains("cluster2")); + + assertTrue(data2.containsKey("/admin/policies/public/default")); + assertFalse(data2.get("/admin/policies/public/default").contains("cluster1")); + assertTrue(data2.get("/admin/policies/public/default").contains("cluster2")); + + assertTrue(data2.containsKey("/admin/policies/pulsar/system")); + assertFalse(data2.get("/admin/policies/pulsar/system").contains("cluster1")); + assertTrue(data2.get("/admin/policies/pulsar/system").contains("cluster2")); } } From 9bcbb206cc7a17a6f23bb61c7a5217eb57c154b7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 12 Nov 2024 17:34:06 +0800 Subject: [PATCH 131/327] [improve][broker] Exclude system topics from namespace level publish and dispatch rate limiting (#23589) --- .../pulsar/broker/service/AbstractTopic.java | 6 ++- .../service/DisabledPublishRateLimiter.java | 44 +++++++++++++++++++ .../service/persistent/SystemTopic.java | 20 +++++++++ .../service/TopicPublishRateThrottleTest.java | 18 ++++++++ .../api/MessageDispatchThrottlingTest.java | 39 ++++++++++++++++ 5 files changed, 125 insertions(+), 2 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DisabledPublishRateLimiter.java 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 96ea2004be8d7..69a38bc50de9d 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 @@ -283,6 +283,10 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { if (log.isDebugEnabled()) { log.debug("[{}]updateTopicPolicyByNamespacePolicy,data={}", topic, namespacePolicies); } + if (!isSystemTopic()) { + updateNamespacePublishRate(namespacePolicies, brokerService.getPulsar().getConfig().getClusterName()); + updateNamespaceDispatchRate(namespacePolicies, brokerService.getPulsar().getConfig().getClusterName()); + } topicPolicies.getRetentionPolicies().updateNamespaceValue(namespacePolicies.retention_policies); topicPolicies.getCompactionThreshold().updateNamespaceValue(namespacePolicies.compaction_threshold); topicPolicies.getReplicationClusters().updateNamespaceValue( @@ -305,7 +309,6 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { topicPolicies.getDeduplicationEnabled().updateNamespaceValue(namespacePolicies.deduplicationEnabled); topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateNamespaceValue( namespacePolicies.deduplicationSnapshotIntervalSeconds); - updateNamespacePublishRate(namespacePolicies, brokerService.getPulsar().getConfig().getClusterName()); topicPolicies.getDelayedDeliveryEnabled().updateNamespaceValue( Optional.ofNullable(namespacePolicies.delayed_delivery_policies) .map(DelayedDeliveryPolicies::isActive).orElse(null)); @@ -326,7 +329,6 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { updateNamespaceSubscriptionDispatchRate(namespacePolicies, brokerService.getPulsar().getConfig().getClusterName()); updateSchemaCompatibilityStrategyNamespaceValue(namespacePolicies); - updateNamespaceDispatchRate(namespacePolicies, brokerService.getPulsar().getConfig().getClusterName()); topicPolicies.getSchemaValidationEnforced().updateNamespaceValue(namespacePolicies.schema_validation_enforced); topicPolicies.getEntryFilters().updateNamespaceValue(namespacePolicies.entryFilters); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DisabledPublishRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DisabledPublishRateLimiter.java new file mode 100644 index 0000000000000..372918e9d3209 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DisabledPublishRateLimiter.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.PublishRate; + +public class DisabledPublishRateLimiter implements PublishRateLimiter { + + public static final DisabledPublishRateLimiter INSTANCE = new DisabledPublishRateLimiter(); + + private DisabledPublishRateLimiter() {} + + @Override + public void handlePublishThrottling(Producer producer, int numOfMessages, long msgSizeInBytes) { + + } + + @Override + public void update(Policies policies, String clusterName) { + + } + + @Override + public void update(PublishRate maxPublishRate) { + + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java index f2cec2138a3a0..8feb432a08001 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java @@ -19,15 +19,20 @@ package org.apache.pulsar.broker.service.persistent; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; +import javax.annotation.Nonnull; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.DisabledPublishRateLimiter; +import org.apache.pulsar.broker.service.PublishRateLimiter; import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.EntryFilters; +import org.apache.pulsar.common.policies.data.Policies; public class SystemTopic extends PersistentTopic { @@ -111,4 +116,19 @@ public EntryFilters getEntryFiltersPolicy() { public List getEntryFilters() { return null; } + + @Override + public PublishRateLimiter getBrokerPublishRateLimiter() { + return DisabledPublishRateLimiter.INSTANCE; + } + + @Override + public void updateResourceGroupLimiter(@Nonnull Policies namespacePolicies) { + // nothing todo. + } + + @Override + public Optional getBrokerDispatchRateLimiter() { + return Optional.empty(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPublishRateThrottleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPublishRateThrottleTest.java index 721d049342552..554bc754c5c43 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPublishRateThrottleTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPublishRateThrottleTest.java @@ -20,6 +20,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.common.policies.data.PublishRate; import org.apache.pulsar.broker.qos.AsyncTokenBucket; @@ -73,6 +75,22 @@ public void testProducerBlockedByPrecisTopicPublishRateLimiting() throws Excepti } } + @Test + public void testSystemTopicPublishNonBlock() throws Exception { + super.baseSetup(); + PublishRate publishRate = new PublishRate(1,10); + admin.namespaces().setPublishRate("prop/ns-abc", publishRate); + final String topic = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/tp"); + PulsarAdmin admin1 = PulsarAdmin.builder().serviceHttpUrl(brokerUrl != null + ? brokerUrl.toString() : brokerUrlTls.toString()).readTimeout(5, TimeUnit.SECONDS).build(); + admin1.topics().createNonPartitionedTopic(topic); + admin1.topicPolicies().setDeduplicationStatus(topic, true); + admin1.topicPolicies().setDeduplicationStatus(topic, false); + // cleanup. + admin.namespaces().removePublishRate("prop/ns-abc"); + admin1.close(); + } + @Test public void testPrecisTopicPublishRateLimitingProduceRefresh() throws Exception { PublishRate publishRate = new PublishRate(1,10); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index 360d27f64133d..a544c7e13bc83 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -48,6 +49,8 @@ import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.PublishRate; +import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.awaitility.Awaitility; @@ -214,6 +217,42 @@ public void testMessageRateDynamicallyChange() throws Exception { producer.close(); } + @SuppressWarnings("deprecation") + @Test + public void testSystemTopicDeliveryNonBlock() throws Exception { + final String namespace = "my-property/throttling_ns"; + admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); + final String topicName = "persistent://" + namespace + "/" + UUID.randomUUID().toString().replaceAll("-", ""); + admin.topics().createNonPartitionedTopic(topicName); + // Set a rate limitation. + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(1) + .dispatchThrottlingRateInByte(-1) + .ratePeriodInSecond(360) + .build(); + admin.namespaces().setDispatchRate(namespace, dispatchRate); + + // Verify the limitation does not take effect. in other words, the topic policies should takes effect. + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + admin.topicPolicies().setPublishRate(topicName, new PublishRate(1000, 1000)); + Awaitility.await().untilAsserted(() -> { + assertNotNull(persistentTopic.getHierarchyTopicPolicies().getPublishRate().getTopicValue()); + }); + admin.topicPolicies().setRetention(topicName, new RetentionPolicies(1000, 1000)); + Awaitility.await().untilAsserted(() -> { + assertNotNull(persistentTopic.getHierarchyTopicPolicies().getRetentionPolicies().getTopicValue()); + }); + admin.topicPolicies().setMessageTTL(topicName, 1000); + Awaitility.await().untilAsserted(() -> { + assertNotNull(persistentTopic.getHierarchyTopicPolicies().getMessageTTLInSeconds().getTopicValue()); + }); + + // cleanup. + admin.topics().delete(topicName); + admin.namespaces().removeDispatchRate(namespace); + } + /** * verify: consumer should not receive all messages due to message-rate throttling * From 9d65a85d6fafbc5f5534caef9b20a808cb5e4d26 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Wed, 13 Nov 2024 17:51:55 +0800 Subject: [PATCH 132/327] [improve] [broker] replace HashMap with inner implementation ConcurrentLongLongPairHashMap in Negative Ack Tracker. (#23582) --- .../pulsar/client/impl/NegativeAcksTest.java | 4 +-- .../client/impl/NegativeAcksTracker.java | 34 ++++++++++++++----- .../pulsar/client/impl/ConsumerImplTest.java | 2 +- 3 files changed, 29 insertions(+), 11 deletions(-) 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 a41b7f05a8eb3..b372ecabc5de4 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 @@ -311,7 +311,7 @@ public void testNegativeAcksDeleteFromUnackedTracker() throws Exception { // negative topic message id consumer.negativeAcknowledge(topicMessageId); NegativeAcksTracker negativeAcksTracker = consumer.getNegativeAcksTracker(); - assertEquals(negativeAcksTracker.getNackedMessagesCount().orElse(-1).intValue(), 1); + assertEquals(negativeAcksTracker.getNackedMessagesCount().orElse((long) -1).longValue(), 1L); assertEquals(unAckedMessageTracker.size(), 0); negativeAcksTracker.close(); // negative batch message id @@ -319,7 +319,7 @@ public void testNegativeAcksDeleteFromUnackedTracker() throws Exception { consumer.negativeAcknowledge(batchMessageId); consumer.negativeAcknowledge(batchMessageId2); consumer.negativeAcknowledge(batchMessageId3); - assertEquals(negativeAcksTracker.getNackedMessagesCount().orElse(-1).intValue(), 1); + assertEquals(negativeAcksTracker.getNackedMessagesCount().orElse((long) -1).longValue(), 1L); assertEquals(unAckedMessageTracker.size(), 0); negativeAcksTracker.close(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java index d6b86e3593dc2..e1724ebb85cda 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java @@ -23,22 +23,23 @@ import io.netty.util.Timeout; import io.netty.util.Timer; import java.io.Closeable; -import java.util.HashMap; import java.util.HashSet; import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.RedeliveryBackoff; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; class NegativeAcksTracker implements Closeable { private static final Logger log = LoggerFactory.getLogger(NegativeAcksTracker.class); - private HashMap nackedMessages = null; + private ConcurrentLongLongPairHashMap nackedMessages = null; private final ConsumerBase consumer; private final Timer timer; @@ -50,6 +51,7 @@ class NegativeAcksTracker implements Closeable { // Set a min delay to allow for grouping nacks within a single batch private static final long MIN_NACK_DELAY_NANOS = TimeUnit.MILLISECONDS.toNanos(100); + private static final long NON_PARTITIONED_TOPIC_PARTITION_INDEX = Long.MAX_VALUE; public NegativeAcksTracker(ConsumerBase consumer, ConsumerConfigurationData conf) { this.consumer = consumer; @@ -75,15 +77,21 @@ private synchronized void triggerRedelivery(Timeout t) { // Group all the nacked messages into one single re-delivery request Set messagesToRedeliver = new HashSet<>(); long now = System.nanoTime(); - nackedMessages.forEach((msgId, timestamp) -> { + nackedMessages.forEach((ledgerId, entryId, partitionIndex, timestamp) -> { if (timestamp < now) { + MessageId msgId = new MessageIdImpl(ledgerId, entryId, + // need to covert non-partitioned topic partition index to -1 + (int) (partitionIndex == NON_PARTITIONED_TOPIC_PARTITION_INDEX ? -1 : partitionIndex)); addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); messagesToRedeliver.add(msgId); } }); if (!messagesToRedeliver.isEmpty()) { - messagesToRedeliver.forEach(nackedMessages::remove); + for (MessageId messageId : messagesToRedeliver) { + nackedMessages.remove(((MessageIdImpl) messageId).getLedgerId(), + ((MessageIdImpl) messageId).getEntryId()); + } consumer.onNegativeAcksSend(messagesToRedeliver); log.info("[{}] {} messages will be re-delivered", consumer, messagesToRedeliver.size()); consumer.redeliverUnacknowledgedMessages(messagesToRedeliver); @@ -102,7 +110,10 @@ public synchronized void add(Message message) { private synchronized void add(MessageId messageId, int redeliveryCount) { if (nackedMessages == null) { - nackedMessages = new HashMap<>(); + nackedMessages = ConcurrentLongLongPairHashMap.newBuilder() + .autoShrink(true) + .concurrencyLevel(1) + .build(); } long backoffNs; @@ -111,7 +122,14 @@ private synchronized void add(MessageId messageId, int redeliveryCount) { } else { backoffNs = nackDelayNanos; } - nackedMessages.put(MessageIdAdvUtils.discardBatch(messageId), System.nanoTime() + backoffNs); + MessageIdAdv messageIdAdv = MessageIdAdvUtils.discardBatch(messageId); + // ConcurrentLongLongPairHashMap requires the key and value >=0. + // partitionIndex is -1 if the message is from a non-partitioned topic, but we don't use + // partitionIndex actually, so we can set it to Long.MAX_VALUE in the case of non-partitioned topic to + // avoid exception from ConcurrentLongLongPairHashMap. + nackedMessages.put(messageIdAdv.getLedgerId(), messageIdAdv.getEntryId(), + messageIdAdv.getPartitionIndex() >= 0 ? messageIdAdv.getPartitionIndex() : + NON_PARTITIONED_TOPIC_PARTITION_INDEX, System.nanoTime() + backoffNs); if (this.timeout == null) { // Schedule a task and group all the redeliveries for same period. Leave a small buffer to allow for @@ -121,8 +139,8 @@ private synchronized void add(MessageId messageId, int redeliveryCount) { } @VisibleForTesting - Optional getNackedMessagesCount() { - return Optional.ofNullable(nackedMessages).map(HashMap::size); + Optional getNackedMessagesCount() { + return Optional.ofNullable(nackedMessages).map(ConcurrentLongLongPairHashMap::size); } @Override diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java index 0c47d17098eb9..e62958eb96887 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java @@ -208,7 +208,7 @@ public void testClose() { Exception checkException = null; try { if (consumer != null) { - consumer.negativeAcknowledge(new MessageIdImpl(-1, -1, -1)); + consumer.negativeAcknowledge(new MessageIdImpl(0, 0, -1)); consumer.close(); } } catch (Exception e) { From 04c80f17d4f48bc0a2062d9cc0e90946fcc145ad Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 13 Nov 2024 09:54:39 +0000 Subject: [PATCH 133/327] [fix][sec] Upgrade to Netty 4.1.115.Final to address CVE-2024-47535 (#23596) --- 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 bd467c7285e91..9fb953929c3c9 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -47,7 +47,7 @@ 4.1 10.14.2 3.1.2 - 4.1.104.Final + 4.1.115.Final 4.2.3 32.1.2-jre 1.10.12 diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 9847b5fec438d..14decd645b165 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -293,33 +293,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.113.Final.jar - - io.netty-netty-codec-4.1.113.Final.jar - - io.netty-netty-codec-dns-4.1.113.Final.jar - - io.netty-netty-codec-http-4.1.113.Final.jar - - io.netty-netty-codec-http2-4.1.113.Final.jar - - io.netty-netty-codec-socks-4.1.113.Final.jar - - io.netty-netty-codec-haproxy-4.1.113.Final.jar - - io.netty-netty-common-4.1.113.Final.jar - - io.netty-netty-handler-4.1.113.Final.jar - - io.netty-netty-handler-proxy-4.1.113.Final.jar - - io.netty-netty-resolver-4.1.113.Final.jar - - io.netty-netty-resolver-dns-4.1.113.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.113.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.113.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.113.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.113.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.113.Final.jar - - io.netty-netty-transport-native-epoll-4.1.113.Final-linux-aarch_64.jar - - io.netty-netty-transport-native-epoll-4.1.113.Final-linux-x86_64.jar - - io.netty-netty-transport-native-unix-common-4.1.113.Final.jar - - io.netty-netty-tcnative-boringssl-static-2.0.66.Final.jar - - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-linux-aarch_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-linux-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-osx-aarch_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-osx-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-windows-x86_64.jar - - io.netty-netty-tcnative-classes-2.0.66.Final.jar + - io.netty-netty-buffer-4.1.115.Final.jar + - io.netty-netty-codec-4.1.115.Final.jar + - io.netty-netty-codec-dns-4.1.115.Final.jar + - io.netty-netty-codec-http-4.1.115.Final.jar + - io.netty-netty-codec-http2-4.1.115.Final.jar + - io.netty-netty-codec-socks-4.1.115.Final.jar + - io.netty-netty-codec-haproxy-4.1.115.Final.jar + - io.netty-netty-common-4.1.115.Final.jar + - io.netty-netty-handler-4.1.115.Final.jar + - io.netty-netty-handler-proxy-4.1.115.Final.jar + - io.netty-netty-resolver-4.1.115.Final.jar + - io.netty-netty-resolver-dns-4.1.115.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.115.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.115.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.115.Final.jar + - io.netty-netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar + - io.netty-netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.115.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.incubator-netty-incubator-transport-classes-io_uring-0.0.24.Final.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 9befa2a803d7f..156690f03f249 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -347,35 +347,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.113.Final.jar - - netty-codec-4.1.113.Final.jar - - netty-codec-dns-4.1.113.Final.jar - - netty-codec-http-4.1.113.Final.jar - - netty-codec-socks-4.1.113.Final.jar - - netty-codec-haproxy-4.1.113.Final.jar - - netty-common-4.1.113.Final.jar - - netty-handler-4.1.113.Final.jar - - netty-handler-proxy-4.1.113.Final.jar - - netty-resolver-4.1.113.Final.jar - - netty-resolver-dns-4.1.113.Final.jar - - netty-transport-4.1.113.Final.jar - - netty-transport-classes-epoll-4.1.113.Final.jar - - netty-transport-native-epoll-4.1.113.Final-linux-aarch_64.jar - - netty-transport-native-epoll-4.1.113.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.113.Final.jar - - netty-tcnative-boringssl-static-2.0.66.Final.jar - - netty-tcnative-boringssl-static-2.0.66.Final-linux-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.66.Final-linux-x86_64.jar - - netty-tcnative-boringssl-static-2.0.66.Final-osx-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.66.Final-osx-x86_64.jar - - netty-tcnative-boringssl-static-2.0.66.Final-windows-x86_64.jar - - netty-tcnative-classes-2.0.66.Final.jar + - netty-buffer-4.1.115.Final.jar + - netty-codec-4.1.115.Final.jar + - netty-codec-dns-4.1.115.Final.jar + - netty-codec-http-4.1.115.Final.jar + - netty-codec-socks-4.1.115.Final.jar + - netty-codec-haproxy-4.1.115.Final.jar + - netty-common-4.1.115.Final.jar + - netty-handler-4.1.115.Final.jar + - netty-handler-proxy-4.1.115.Final.jar + - netty-resolver-4.1.115.Final.jar + - netty-resolver-dns-4.1.115.Final.jar + - netty-transport-4.1.115.Final.jar + - netty-transport-classes-epoll-4.1.115.Final.jar + - netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.115.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-incubator-transport-classes-io_uring-0.0.24.Final.jar - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.113.Final.jar - - netty-resolver-dns-native-macos-4.1.113.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.113.Final-osx-x86_64.jar + - netty-resolver-dns-classes-macos-4.1.115.Final.jar + - netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.115.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 ebdf3492d3f3b..1e93bbdbaee21 100644 --- a/pom.xml +++ b/pom.xml @@ -146,7 +146,7 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.7.1 - 4.1.113.Final + 4.1.115.Final 0.0.24.Final 9.4.56.v20240826 2.5.2 From 0f934f2f0729ba90b202582710ba36f2e32cefb3 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Wed, 13 Nov 2024 18:25:11 +0800 Subject: [PATCH 134/327] [fix][client] The partitionedProducer maxPendingMessages always is 0 (#23593) --- .../service/PersistentTopicE2ETest.java | 44 +++++++++++++++++++ .../client/impl/PartitionedProducerImpl.java | 15 +++++-- .../impl/PartitionedProducerImplTest.java | 38 ++++++++++++++++ 3 files changed, 94 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java index 4d79e7ccdf0d1..8e3b920e002b6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java @@ -75,6 +75,7 @@ import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.PartitionedProducerImpl; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; @@ -1340,6 +1341,49 @@ public void testProducerQueueFullBlocking() throws Exception { setup(); } + @Test + public void testProducerQueueFullBlockingWithPartitionedTopic() throws Exception { + final String topicName = "persistent://prop/ns-abc/topic-xyzx2"; + admin.topics().createPartitionedTopic(topicName, 2); + + @Cleanup + PulsarClient client = PulsarClient.builder().serviceUrl(brokerUrl.toString()).build(); + + // 1. Producer connect + PartitionedProducerImpl producer = (PartitionedProducerImpl) client.newProducer() + .topic(topicName) + .maxPendingMessages(1) + .blockIfQueueFull(true) + .sendTimeout(1, TimeUnit.SECONDS) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + + // 2. Stop broker + cleanup(); + + // 2. producer publish messages + long startTime = System.nanoTime(); + producer.sendAsync("msg".getBytes()); + + // Verify thread was not blocked + long delayNs = System.nanoTime() - startTime; + assertTrue(delayNs < TimeUnit.SECONDS.toNanos(1)); + + // Next send operation must block, until all the messages in the queue expire + startTime = System.nanoTime(); + producer.sendAsync("msg".getBytes()); + delayNs = System.nanoTime() - startTime; + assertTrue(delayNs > TimeUnit.MILLISECONDS.toNanos(500)); + assertTrue(delayNs < TimeUnit.MILLISECONDS.toNanos(1500)); + + // 4. producer disconnect + producer.close(); + + // 5. Restart broker + setup(); + } + @Test public void testProducerQueueFullNonBlocking() throws Exception { final String topicName = "persistent://prop/ns-abc/topic-xyzx"; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java index 2dc826d9e3af3..903a1beaaeee3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java @@ -19,6 +19,8 @@ package org.apache.pulsar.client.impl; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_MAX_PENDING_MESSAGES; +import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import io.netty.util.Timeout; @@ -84,9 +86,16 @@ public PartitionedProducerImpl(PulsarClientImpl client, String topic, ProducerCo : null; // MaxPendingMessagesAcrossPartitions doesn't support partial partition such as SinglePartition correctly - int maxPendingMessages = Math.min(conf.getMaxPendingMessages(), - conf.getMaxPendingMessagesAcrossPartitions() / numPartitions); - conf.setMaxPendingMessages(maxPendingMessages); + int maxPendingMessages = conf.getMaxPendingMessages(); + int maxPendingMessagesAcrossPartitions = conf.getMaxPendingMessagesAcrossPartitions(); + if (maxPendingMessagesAcrossPartitions != DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS) { + int maxPendingMsgsForOnePartition = maxPendingMessagesAcrossPartitions / numPartitions; + maxPendingMessages = (maxPendingMessages == DEFAULT_MAX_PENDING_MESSAGES) + ? maxPendingMsgsForOnePartition + : Math.min(maxPendingMessages, maxPendingMsgsForOnePartition); + conf.setMaxPendingMessages(maxPendingMessages); + } + final List indexList; if (conf.isLazyStartPartitionedProducers() diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java index f96d2e2e0b0e9..0ce95dc1264a7 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java @@ -272,6 +272,44 @@ public void testGetNumOfPartitions() throws Exception { assertEquals(producerImpl.getNumOfPartitions(), 0); } + @Test + public void testMaxPendingQueueSize() throws Exception { + String topicName = "test-max-pending-queue-size"; + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setServiceUrl("pulsar://localhost:6650"); + conf.setStatsIntervalSeconds(100); + + ThreadFactory threadFactory = new DefaultThreadFactory("client-test-stats", Thread.currentThread().isDaemon()); + @Cleanup("shutdownGracefully") + EventLoopGroup eventLoopGroup = EventLoopUtil.newEventLoopGroup(conf.getNumIoThreads(), false, threadFactory); + + @Cleanup + PulsarClientImpl clientImpl = new PulsarClientImpl(conf, eventLoopGroup); + + // Test set maxPendingMessage to 10 + ProducerConfigurationData producerConfData = new ProducerConfigurationData(); + producerConfData.setMessageRoutingMode(MessageRoutingMode.CustomPartition); + producerConfData.setCustomMessageRouter(new CustomMessageRouter()); + producerConfData.setMaxPendingMessages(10); + PartitionedProducerImpl partitionedProducerImpl = new PartitionedProducerImpl( + clientImpl, topicName, producerConfData, 1, null, null, null); + assertEquals(partitionedProducerImpl.getConfiguration().getMaxPendingMessages(), 10); + + // Test set MaxPendingMessagesAcrossPartitions=5 + producerConfData.setMaxPendingMessages(ProducerConfigurationData.DEFAULT_MAX_PENDING_MESSAGES); + producerConfData.setMaxPendingMessagesAcrossPartitions(5); + partitionedProducerImpl = new PartitionedProducerImpl( + clientImpl, topicName, producerConfData, 1, null, null, null); + assertEquals(partitionedProducerImpl.getConfiguration().getMaxPendingMessages(), 5); + + // Test set maxPendingMessage=10 and MaxPendingMessagesAcrossPartitions=10 with 2 partitions + producerConfData.setMaxPendingMessages(10); + producerConfData.setMaxPendingMessagesAcrossPartitions(10); + partitionedProducerImpl = new PartitionedProducerImpl( + clientImpl, topicName, producerConfData, 2, null, null, null); + assertEquals(partitionedProducerImpl.getConfiguration().getMaxPendingMessages(), 5); + } + @Test public void testOnTopicsExtended() throws Exception { From 5d924bfb451bccdad0c1dc9a22c6dde49d65a47d Mon Sep 17 00:00:00 2001 From: grayson <916028390@qq.com> Date: Thu, 14 Nov 2024 05:07:01 +0800 Subject: [PATCH 135/327] Enabling DNS retryOnTimeout with TCP in DnsNameResolver (#23590) --- .../java/org/apache/pulsar/client/impl/ConnectionPool.java | 4 +++- .../java/org/apache/pulsar/proxy/server/ProxyService.java | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java index 9be734bee6173..1eb603e28f571 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java @@ -188,7 +188,9 @@ public ConnectionPool(InstrumentProvider instrumentProvider, private static AddressResolver createAddressResolver(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder() - .traceEnabled(true).channelType(EventLoopUtil.getDatagramChannelClass(eventLoopGroup)); + .traceEnabled(true) + .channelType(EventLoopUtil.getDatagramChannelClass(eventLoopGroup)) + .socketChannelType(EventLoopUtil.getClientSocketChannelClass(eventLoopGroup), true); if (conf.getDnsLookupBindAddress() != null) { InetSocketAddress addr = new InetSocketAddress(conf.getDnsLookupBindAddress(), conf.getDnsLookupBindPort()); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index 4ee15fd7124a6..11afb68398e19 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -179,7 +179,8 @@ public ProxyService(ProxyConfiguration proxyConfig, this.authenticationService = authenticationService; DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder() - .channelType(EventLoopUtil.getDatagramChannelClass(workerGroup)); + .channelType(EventLoopUtil.getDatagramChannelClass(workerGroup)) + .socketChannelType(EventLoopUtil.getClientSocketChannelClass(workerGroup), true); DnsResolverUtil.applyJdkDnsCacheSettings(dnsNameResolverBuilder); dnsAddressResolverGroup = new DnsAddressResolverGroup(dnsNameResolverBuilder); From 22cfa5428ef8a4fc6cfc0225182724012f0972c2 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 14 Nov 2024 12:40:53 +0800 Subject: [PATCH 136/327] [improve][pip] PIP-391: Enable batch index ACK by default (#23567) --- ...p-391-enable-batch-index-ack-by-default.md | 110 ++++++++++++++++++ 1 file changed, 110 insertions(+) create mode 100644 pip/pip-391-enable-batch-index-ack-by-default.md diff --git a/pip/pip-391-enable-batch-index-ack-by-default.md b/pip/pip-391-enable-batch-index-ack-by-default.md new file mode 100644 index 0000000000000..850ab4856abd5 --- /dev/null +++ b/pip/pip-391-enable-batch-index-ack-by-default.md @@ -0,0 +1,110 @@ +## Background + +### Default Approach to Acknowledge Batched Messages + +A batched message is part of a batch stored in a BookKeeper entry. For instance, given a batch containing N messages, these N messages are stored in the same entry. The message IDs of these messages share the same ledger ID and entry ID, differing only in their batch indexes. + +Consider the following code example: + +```java +for (int i = 0; i < 3; i++) { + producer.newMessage().value("msg-" + i).sendAsync(); +} +producer.flush(); +final var ids = new ArrayList(); +for (int i = 0; i < 3; i++) { + ids.add(consumer.receive().getMessageId()); +} +consumer.acknowledge(ids.get(0)); +``` + +Here, `ids` are the message IDs of the 3 messages in the same batch, with batch indexes 0, 1, and 2. These message IDs share the same bit set object with 3 bits set (represented as "111"). The side effect of `acknowledge(ids.get(0))` is to set the bit of batch index 0, changing the bit set to "011". Only when all bits are cleared will the consumer send an ACK request to the broker, indicating that the entry of this batch has been acknowledged. + +If the `consumer` restarts after acknowledging `ids.get(0)`, the broker will dispatch this entry again, causing the `consumer` to receive duplicate messages: + +```java +consumer.acknowledge(ids.get(0)); // the batch index of ids.get(0) is 0 +consumer.close(); +consumer = pulsarClient.newConsumer(Schema.STRING).topic(topic).subscriptionName("sub") + .enableBatchIndexAcknowledgment(true).subscribe(); +final var msg = consumer.receive(); // msg.getValue() will be "msg-0" and the batch index is 0 +``` + +This behavior can be confusing. Users might assume that the message has been acknowledged and will not be received again. + +Generally, if every received message is acknowledged, this issue does not arise because a consumer must be able to receive a whole batch. However, in some common use cases, this behavior can cause issues. + +#### Case 1: Message ID Comes from Deserialization + +Pulsar supports serializing the message ID into bytes via the `toByteArray()` method and recovering the message ID from the raw bytes via `fromByteArrayWithTopic()`. This is useful when storing the message ID in a database or other storage. + +However, it is impossible to recover the bit set for message IDs in the same batch because they must share the same bit set object. The existing implementation creates a bit set for each batched message ID, which is meaningless. After acknowledging all of them, each message ID will have a separate bit set with only 1 bit cleared, e.g.: +- Bit set of `id0`: 011 +- Bit set of `id1` 101 +- Bit set of `id2`: 110 + +To solve this issue, a stateful message ID deserializer can be provided, which remembers if all message IDs in the same batch have been deserialized and only creates an ack set for the first message ID in the batch. However, this API would be complicated and not user-friendly as well. It would still not work for cases where serialized message IDs in the same batch are stored in different places. + +#### Case 2: Negative Acknowledgment + +When a consumer fails to process a message, Pulsar provides a `negativeAcknowledge` method to trigger the redelivery of that message after some time, determined by the `negativeAckRedeliveryDelay` config. + +```java +consumer.acknowledge(ids.get(0)); +consumer.acknowledge(ids.get(2)); +consumer.negativeAcknowledge(ids.get(1)); +final var msg = consumer.receive(); +``` + +In this example, it is expected that `msg`s batch index is 1. + +Unfortunately, with the default config, `msg`'s batch index will be 0 because Pulsar will dispatch the whole batch while the consumer is not able to filter out the acknowledged single messages whose message ID is `ids.get(0)` or `ids.get(2)`. + +#### Summary + +The default behavior of acknowledging batched messages is not user-friendly and might cause unexpected issues. When these issues arise, it is hard to understand the root cause without knowing the details of the implementation. + +## Motivation + +[PIP-54](https://github.com/apache/pulsar/wiki/PIP-54:-Support-acknowledgment-at-batch-index-level) introduces the ability to acknowledge messages at the batch index level. Two configurations control this behavior: +- Client side: `ConsumerBuilder#enableBatchIndexAcknowledgment`: false by default. When true, the client sends the batch index ACK request to the broker. +- Broker side: `acknowledgmentAtBatchIndexLevelEnabled`: false by default. When true, the broker handles the batch index ACK request from the client. + +When both are enabled, the issues with the default ACK behaviors are resolved. However, these configurations are not enabled by default due to concerns about compatibility or performance: +1. The client with batch index ACK enabled cannot work with brokers before version 2.6.0. +2. The broker needs to maintain the ack set for each batched message ID in memory and persist it somewhere. +3. When `isAckReceiptEnabled` is true, each `acknowledge` call triggers a request to the broker and waits for the response. + +For issue 1, it is not a significant concern because version 2.6.0 is very old. Even for old brokers, the behavior remains the same with batch index ACK disabled. + +For issue 2, no performance issues have been reported with batch index ACK enabled. Generally, correctness should have higher priority than performance. It is more worthwhile to improve the performance of an API with correct semantics than to fear performance issues and provide a confusing API by default. + +For issue 3, the number of RPCs can be reduced by leveraging the `acknowledge` API that accepts a list of message IDs: + +```java +void acknowledge(List messageIdList) throws PulsarClientException; +``` + +Additionally, the `isAckReceiptEnabled` config is disabled by default, meaning even the synchronous `acknowledge` API does not require an ACK request to be sent for this message ID. The acknowledgments are cached and grouped into a single ACK request after some time, determined by the `acknowledgmentGroupTime` and `maxAcknowledgmentGroupSize` configs. + +## Goals + +Enable the batch index ACK feature by default for both client and broker. + +## High-Level Design + +For Pulsar 4.x, enable `acknowledgmentAtBatchIndexLevelEnabled` and `ConsumerBuilder#enableBatchIndexAcknowledgment` by default. + +For Pulsar 5.x, deprecate the `ConsumerBuilder#enableBatchIndexAcknowledgment` config and enforce batch index ACK regardless of its setting. + +## Backward & Forward Compatibility + +- Batch index ACK does not work on Pulsar brokers earlier than version 2.6.0. +- For older versions that do not implement this PIP, `acknowledgmentAtBatchIndexLevelEnabled` must be configured to true manually. + +## General Notes + +## Links + +* Mailing List discussion thread: https://lists.apache.org/thread/y19s4tn1xtbh8lxbgcf0pc8yy6xs2k25 +* Mailing List voting thread: https://lists.apache.org/thread/6l2th8fwvx9gtljcdov6c1mz7d7oopt9 From 0b0eef905c3a6ebe8a6fbb284743d98c8e97d5b8 Mon Sep 17 00:00:00 2001 From: Zhengke Zhou Date: Thu, 14 Nov 2024 18:27:10 +0800 Subject: [PATCH 137/327] [improve][test] Clarify method signatures in Bookkeeper mock client (#23598) --- .../client/BookKeeperTestClient.java | 26 +++++++++---------- .../client/PulsarMockBookKeeper.java | 4 +-- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/BookKeeperTestClient.java b/testmocks/src/main/java/org/apache/bookkeeper/client/BookKeeperTestClient.java index dd33c2c4532bf..6aa949d7524c0 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/BookKeeperTestClient.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/BookKeeperTestClient.java @@ -71,14 +71,14 @@ public BookieClient getBookieClient() { return bookieClient; } - public Future waitForReadOnlyBookie(BookieId b) + public Future waitForReadOnlyBookie(BookieId id) throws Exception { - return waitForBookieInSet(b, false); + return waitForBookieInSet(id, false); } - public Future waitForWritableBookie(BookieId b) + public Future waitForWritableBookie(BookieId id) throws Exception { - return waitForBookieInSet(b, true); + return waitForBookieInSet(id, true); } /** @@ -86,23 +86,23 @@ public Future waitForWritableBookie(BookieId b) * or the read only set of bookies. Also ensure that it doesn't exist * in the other set before completing. */ - private Future waitForBookieInSet(BookieId b, + private Future waitForBookieInSet(BookieId id, boolean writable) throws Exception { log.info("Wait for {} to become {}", - b, writable ? "writable" : "readonly"); + id, writable ? "writable" : "readonly"); CompletableFuture readOnlyFuture = new CompletableFuture<>(); CompletableFuture writableFuture = new CompletableFuture<>(); RegistrationListener readOnlyListener = (bookies) -> { - boolean contains = bookies.getValue().contains(b); - if ((!writable && contains) || (writable && !contains)) { + boolean containsId = bookies.getValue().contains(id); + if ((!writable && containsId) || (writable && !containsId)) { readOnlyFuture.complete(null); } }; RegistrationListener writableListener = (bookies) -> { - boolean contains = bookies.getValue().contains(b); - if ((writable && contains) || (!writable && !contains)) { + boolean containsId = bookies.getValue().contains(id); + if ((writable && containsId) || (!writable && !containsId)) { writableFuture.complete(null); } }; @@ -114,7 +114,7 @@ private Future waitForBookieInSet(BookieId b, return writableFuture .thenCompose(ignored -> getMetadataClientDriver().getRegistrationClient().getReadOnlyBookies()) .thenCompose(readonlyBookies -> { - if (readonlyBookies.getValue().contains(b)) { + if (readonlyBookies.getValue().contains(id)) { // if the bookie still shows up at readonly path, wait for it to disappear return readOnlyFuture; } else { @@ -125,7 +125,7 @@ private Future waitForBookieInSet(BookieId b, return readOnlyFuture .thenCompose(ignored -> getMetadataClientDriver().getRegistrationClient().getWritableBookies()) .thenCompose(writableBookies -> { - if (writableBookies.getValue().contains(b)) { + if (writableBookies.getValue().contains(id)) { // if the bookie still shows up at writable path, wait for it to disappear return writableFuture; } else { @@ -139,7 +139,7 @@ private Future waitForBookieInSet(BookieId b, * Force a read to zookeeper to get list of bookies. * * @throws InterruptedException - * @throws KeeperException + * @throws BKException */ public void readBookiesBlocking() throws InterruptedException, BKException { bookieWatcher.initialBlockingBookieRead(); diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java index 4516cfea01f05..11ec2dec938a8 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -105,13 +105,13 @@ public OrderedExecutor getMainWorkerPool() { } @Override - public LedgerHandle createLedger(DigestType digestType, byte passwd[]) + public LedgerHandle createLedger(DigestType digestType, byte[] passwd) throws BKException, InterruptedException { return createLedger(3, 2, digestType, passwd); } @Override - public LedgerHandle createLedger(int ensSize, int qSize, DigestType digestType, byte passwd[]) + public LedgerHandle createLedger(int ensSize, int qSize, DigestType digestType, byte[] passwd) throws BKException, InterruptedException { return createLedger(ensSize, qSize, qSize, digestType, passwd); } From 89ccb7361b80dfb7897ea2555dfb6e884f62b916 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=90=A7=E6=98=93=E5=AE=A2?= Date: Sun, 17 Nov 2024 02:45:12 +0800 Subject: [PATCH 138/327] [fix][misc] Unable to connect an etcd metastore with recent releases due to jetc-core sharding problem (#23604) Co-authored-by: Lari Hotari --- distribution/server/pom.xml | 1 - jetcd-core-shaded/pom.xml | 49 +------------------------------------ pom.xml | 1 - pulsar-broker/pom.xml | 1 - pulsar-metadata/pom.xml | 1 - 5 files changed, 1 insertion(+), 52 deletions(-) diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 265d21a672cbe..c8815fdd8da8d 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -49,7 +49,6 @@ ${project.groupId} jetcd-core-shaded ${project.version} - shaded diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index 4cafcc1a44b77..b0e6e99052599 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -74,6 +74,7 @@ + ${project.artifactId}-${project.version} org.apache.maven.plugins @@ -141,54 +142,6 @@ ${project.basedir}/dependency-reduced-pom.xml - - true - shaded - - - - - - - org.codehaus.mojo - build-helper-maven-plugin - - - attach-shaded-jar - package - - attach-artifact - - - - - ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar - jar - shaded - - - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - ${maven-antrun-plugin.version} - - - unpack-shaded-jar - package - - run - - - - - diff --git a/pom.xml b/pom.xml index 1e93bbdbaee21..504a4a6b1bb66 100644 --- a/pom.xml +++ b/pom.xml @@ -1108,7 +1108,6 @@ flexible messaging model and an intuitive client API. ${project.groupId} jetcd-core-shaded ${project.version} - shaded io.etcd diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index dab39c0609da5..97ede1f76e969 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -487,7 +487,6 @@ ${project.groupId} jetcd-core-shaded ${project.version} - shaded test diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index e32473cd434c3..229dd5ba610a9 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -125,7 +125,6 @@ ${project.groupId} jetcd-core-shaded ${project.version} - shaded io.grpc From b915f6e73e5b8cfa94c4c55eb930befc682674d4 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 18 Nov 2024 10:09:26 +0800 Subject: [PATCH 139/327] [improve][offload] Use filesystemURI as the storage path (#23591) --- .../FileSystemManagedLedgerOffloader.java | 2 +- .../FileSystemOffloaderLocalFileTest.java | 119 ++++++++++++++++++ .../filesystem_offload_core_site.xml | 48 +++++++ 3 files changed, 168 insertions(+), 1 deletion(-) create mode 100644 tiered-storage/file-system/src/test/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemOffloaderLocalFileTest.java create mode 100644 tiered-storage/file-system/src/test/resources/filesystem_offload_core_site.xml diff --git a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java index 56612adc1ef80..2431e61c91cab 100644 --- a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java +++ b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java @@ -107,7 +107,7 @@ private FileSystemManagedLedgerOffloader(OffloadPoliciesImpl conf, OrderedSchedu this.configuration.setClassLoader(FileSystemLedgerOffloaderFactory.class.getClassLoader()); this.driverName = conf.getManagedLedgerOffloadDriver(); - this.storageBasePath = configuration.get("hadoop.tmp.dir"); + this.storageBasePath = configuration.get("fs.defaultFS"); this.scheduler = scheduler; this.fileSystem = FileSystem.get(configuration); this.assignmentScheduler = OrderedScheduler.newSchedulerBuilder() diff --git a/tiered-storage/file-system/src/test/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemOffloaderLocalFileTest.java b/tiered-storage/file-system/src/test/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemOffloaderLocalFileTest.java new file mode 100644 index 0000000000000..14734b3faca99 --- /dev/null +++ b/tiered-storage/file-system/src/test/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemOffloaderLocalFileTest.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.offload.filesystem.impl; + +import static org.testng.Assert.assertEquals; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.UUID; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.PulsarMockBookKeeper; +import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.client.api.LedgerEntry; +import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.mledger.LedgerOffloaderStats; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; +import org.testng.annotations.Test; + +public class FileSystemOffloaderLocalFileTest { + private OrderedScheduler scheduler = OrderedScheduler.newSchedulerBuilder().numThreads(1).name("offloader").build(); + private LedgerOffloaderStats offloaderStats = LedgerOffloaderStats.create(true, true, scheduler, 60); + + + private String getResourceFilePath(String name) { + return getClass().getClassLoader().getResource(name).getPath(); + } + + @Test + public void testReadWriteWithLocalFileUsingFileSystemURI() throws Exception { + // prepare the offload policies + final String basePath = "/tmp"; + OffloadPoliciesImpl offloadPolicies = new OffloadPoliciesImpl(); + offloadPolicies.setFileSystemURI("file://" + basePath); + offloadPolicies.setManagedLedgerOffloadDriver("filesystem"); + offloadPolicies.setFileSystemProfilePath(getResourceFilePath("filesystem_offload_core_site.xml")); + + // initialize the offloader with the offload policies + var offloader = FileSystemManagedLedgerOffloader.create(offloadPolicies, scheduler, offloaderStats); + + int numberOfEntries = 100; + + // prepare the data in bookkeeper + BookKeeper bk = new PulsarMockBookKeeper(scheduler); + LedgerHandle lh = bk.createLedger(1,1,1, BookKeeper.DigestType.CRC32, "".getBytes()); + for (int i = 0; i < numberOfEntries; i++) { + byte[] entry = ("foobar"+i).getBytes(); + lh.addEntry(entry); + } + lh.close(); + + ReadHandle read = bk.newOpenLedgerOp() + .withLedgerId(lh.getId()) + .withDigestType(DigestType.CRC32) + .withPassword("".getBytes()).execute().get(); + + final String mlName = TopicName.get("testWriteLocalFIle").getPersistenceNamingEncoding(); + Map offloadDriverMetadata = new HashMap<>(); + offloadDriverMetadata.put("ManagedLedgerName", mlName); + + UUID uuid = UUID.randomUUID(); + offloader.offload(read, uuid, offloadDriverMetadata).get(); + ReadHandle toTest = offloader.readOffloaded(read.getId(), uuid, offloadDriverMetadata).get(); + assertEquals(toTest.getLastAddConfirmed(), read.getLastAddConfirmed()); + LedgerEntries toTestEntries = toTest.read(0, numberOfEntries - 1); + LedgerEntries toWriteEntries = read.read(0,numberOfEntries - 1); + Iterator toTestIter = toTestEntries.iterator(); + Iterator toWriteIter = toWriteEntries.iterator(); + while(toTestIter.hasNext()) { + LedgerEntry toWriteEntry = toWriteIter.next(); + LedgerEntry toTestEntry = toTestIter.next(); + + assertEquals(toWriteEntry.getLedgerId(), toTestEntry.getLedgerId()); + assertEquals(toWriteEntry.getEntryId(), toTestEntry.getEntryId()); + assertEquals(toWriteEntry.getLength(), toTestEntry.getLength()); + assertEquals(toWriteEntry.getEntryBuffer(), toTestEntry.getEntryBuffer()); + } + toTestEntries = toTest.read(1, numberOfEntries - 1); + toWriteEntries = read.read(1,numberOfEntries - 1); + toTestIter = toTestEntries.iterator(); + toWriteIter = toWriteEntries.iterator(); + while(toTestIter.hasNext()) { + LedgerEntry toWriteEntry = toWriteIter.next(); + LedgerEntry toTestEntry = toTestIter.next(); + + assertEquals(toWriteEntry.getLedgerId(), toTestEntry.getLedgerId()); + assertEquals(toWriteEntry.getEntryId(), toTestEntry.getEntryId()); + assertEquals(toWriteEntry.getLength(), toTestEntry.getLength()); + assertEquals(toWriteEntry.getEntryBuffer(), toTestEntry.getEntryBuffer()); + } + + // check the file located in the local file system + Path offloadedFilePath = Paths.get(basePath, mlName); + assertEquals(Files.exists(offloadedFilePath), true); + } +} diff --git a/tiered-storage/file-system/src/test/resources/filesystem_offload_core_site.xml b/tiered-storage/file-system/src/test/resources/filesystem_offload_core_site.xml new file mode 100644 index 0000000000000..d26cec2cc60f0 --- /dev/null +++ b/tiered-storage/file-system/src/test/resources/filesystem_offload_core_site.xml @@ -0,0 +1,48 @@ + + + + + fs.defaultFS + + + + hadoop.tmp.dir + pulsar + + + io.file.buffer.size + 4096 + + + io.seqfile.compress.blocksize + 1000000 + + + io.seqfile.compression.type + BLOCK + + + io.map.index.interval + 128 + + + From 27158532de26d40e7a402769e73ddd3be43f0623 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 18 Nov 2024 10:18:30 +0800 Subject: [PATCH 140/327] [fix][broker] Fix failed TokenAuthenticatedProducerConsumerTest (#23602) --- .../client/api/TokenAuthenticatedProducerConsumerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java index 9da3fcbd0edc9..a9a568f1326d7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenAuthenticatedProducerConsumerTest.java @@ -191,14 +191,14 @@ public static Object[][] provider() { } @Test(dataProvider = "provider") - public void testTopicNotFound(boolean useTcpServiceUrl, boolean useCorrectToken) throws Exception { + public void testTenantNotExist(boolean useTcpServiceUrl, boolean useCorrectToken) throws Exception { final var operationTimeoutMs = 10000; final var url = useTcpServiceUrl ? pulsar.getBrokerServiceUrl() : pulsar.getWebServiceAddress(); final var token = useCorrectToken ? ADMIN_TOKEN : USER_TOKEN; @Cleanup final var client = PulsarClient.builder().serviceUrl(url) .operationTimeout(operationTimeoutMs, TimeUnit.MILLISECONDS) .authentication(AuthenticationFactory.token(token)).build(); - final var topic = "my-property/not-exist/tp"; // the namespace does not exist + final var topic = "non-exist/not-exist/tp"; // the namespace does not exist var start = System.currentTimeMillis(); try { client.newProducer().topic(topic).create(); From 387a96dcdd78509547b09f12092104620326aa9c Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Tue, 19 Nov 2024 13:38:30 +0800 Subject: [PATCH 141/327] [feat][misc] Upgrade oxia version to 0.4.9 (#23607) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 14decd645b165..7c66460c21656 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -481,8 +481,8 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-api-0.4.7.jar - - io.streamnative.oxia-oxia-client-0.4.7.jar + - io.streamnative.oxia-oxia-client-api-0.4.9.jar + - io.streamnative.oxia-oxia-client-0.4.9.jar * OpenHFT - net.openhft-zero-allocation-hashing-0.16.jar * Java JSON WebTokens diff --git a/pom.xml b/pom.xml index 504a4a6b1bb66..40afef1241a54 100644 --- a/pom.xml +++ b/pom.xml @@ -251,7 +251,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.7 - 0.4.7 + 0.4.9 2.0 1.10.12 5.5.0 From 895e96853e56b7d9c10c6bd0f3bd6105b664eb14 Mon Sep 17 00:00:00 2001 From: sinan liu Date: Wed, 20 Nov 2024 00:24:46 +0800 Subject: [PATCH 142/327] [improve][ml] Avoid repetitive nested lock for isMessageDeleted in ManagedCursorImpl (#23609) --- .../bookkeeper/mledger/impl/ManagedCursorImpl.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 7c0d13108b1c4..478c6a1b37976 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1569,7 +1569,7 @@ public Set asyncReplayEntries(Set positi Set alreadyAcknowledgedPositions = new HashSet<>(); lock.readLock().lock(); try { - positions.stream().filter(this::isMessageDeleted).forEach(alreadyAcknowledgedPositions::add); + positions.stream().filter(this::internalIsMessageDeleted).forEach(alreadyAcknowledgedPositions::add); } finally { lock.readLock().unlock(); } @@ -2345,7 +2345,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb return; } - if (isMessageDeleted(position)) { + if (internalIsMessageDeleted(position)) { if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { @@ -3543,13 +3543,19 @@ public Position processIndividuallyDeletedMessagesAndGetMarkDeletedPosition( public boolean isMessageDeleted(Position position) { lock.readLock().lock(); try { - return position.compareTo(markDeletePosition) <= 0 - || individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()); + return internalIsMessageDeleted(position); } finally { lock.readLock().unlock(); } } + // When this method is called while the external has already acquired a write lock or a read lock, + // it avoids unnecessary lock nesting. + private boolean internalIsMessageDeleted(Position position) { + return position.compareTo(markDeletePosition) <= 0 + || individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()); + } + //this method will return a copy of the position's ack set @Override public long[] getBatchPositionAckSet(Position position) { From 5338dc99ed398060b03a4ecc5b63849caeae3590 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Wed, 20 Nov 2024 21:00:32 +0800 Subject: [PATCH 143/327] [improve][pip] PIP-392: Enable consistent hashing to select active consumer in partitioned topic for failover subscription (#23583) --- pip/pip-392.md | 97 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 97 insertions(+) create mode 100644 pip/pip-392.md diff --git a/pip/pip-392.md b/pip/pip-392.md new file mode 100644 index 0000000000000..ff5237fdb48e1 --- /dev/null +++ b/pip/pip-392.md @@ -0,0 +1,97 @@ +# PIP-392: Add configuration to enable consistent hashing to select active consumer for partitioned topic + +# Background knowledge + +After [#19502](https://github.com/apache/pulsar/pull/19502) will use consistent hashing to select active consumer for non-partitioned topic + +# Motivation + +Currently, for partitioned topics, the active consumer is selected using the formula [partitionedIndex % consumerSize](https://github.com/apache/pulsar/blob/137df29f85798b00de75460a1acb91c7bc25453f/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java#L129-L130). +This method can lead to uneven distribution of active consumers. + +Consider a scenario with 100 topics named `public/default/topic-{0~100}`, each having `one partition`. +If 10 consumers are created using a `regex` subscription with the `Failover type`, all topic will be assigned to the same consumer(the first connected consumer). This results in an imbalanced distribution of consumers. + +# Goals + +## In Scope +- Address the issue of imbalance for `failover` subscription type consumers in single-partition or few-partition topics. + +## Out of Scope +- Excluding the `exclusive` subscription type. + +It's important to note that both the `modulo algorithm` and the `consistent hashing algorithm` can cause the consumer to be transferred. +This might result in messages being delivered multiple times to consumers, which is a known issue and has been mentioned in the documentation. +https://pulsar.apache.org/docs/4.0.x/concepts-messaging/#failover + +# High Level Design +The solution involves adding a configuration setting that allows users to enable consistent hashing for partitioned topics. +When enabled, the consumer selection process will use consistent hashing instead of the modulo operation. + +The algorithm already exists through [#19502](https://github.com/apache/pulsar/pull/19502) + +In simple terms, the hash algorithm includes the following steps: + +1. Hash Ring Creation: Traverse all consumers and use `consumer name` to calculate a hash ring with 100 virtual nodes. + +[Exist code](https://github.com/apache/pulsar/blob/1b1bd4b610dd768a6908964ef841a6790bb0f4f0/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java#L152-L162) +```java + private NavigableMap makeHashRing(int consumerSize) { + NavigableMap hashRing = new TreeMap<>(); + for (int i = 0; i < consumerSize; i++) { + for (int j = 0; j < CONSUMER_CONSISTENT_HASH_REPLICAS; j++) { + String key = consumers.get(i).consumerName() + j; + int hash = Murmur3_32Hash.getInstance().makeHash(key.getBytes()); + hashRing.put(hash, i); + } + } + return Collections.unmodifiableNavigableMap(hashRing); + } +``` + +2. Consumer Selection: Use the hash of the topic name to select the matching consumer from the hash ring. + +[Exist code](https://github.com/apache/pulsar/blob/1b1bd4b610dd768a6908964ef841a6790bb0f4f0/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java#L146-L150) +```java + private int peekConsumerIndexFromHashRing(NavigableMap hashRing) { + int hash = Murmur3Hash32.getInstance().makeHash(topicName); + Map.Entry ceilingEntry = hashRing.ceilingEntry(hash); + return ceilingEntry != null ? ceilingEntry.getValue() : hashRing.firstEntry().getValue(); + } +``` + +This approach ensures a more even distribution of active consumers across topics, improving load balancing and resource utilization. + +# Detailed Design + +## Design & Implementation Details +Refer to implementation PR: https://github.com/apache/pulsar/pull/23584 + +The implementation is simple. If this activeConsumerFailoverConsistentHashing is enabled, the consistent hashing algorithm is used regardless of whether the topic is partitioned. + +## Public-facing Changes + +If activeConsumerFailoverConsistentHashing is enabled, when users use the failover subscription model, +the `first consumer` will not necessarily consume `P1`, and the `second consumer` will not necessarily consume `P2`. + +As described in the documentation:: https://pulsar.apache.org/docs/4.0.x/concepts-messaging/#failover--partitioned-topics + +Instead, the hash algorithm will determine which consumer consumes which partition. + +### Configuration + +A new configuration field will be added: + +```java +@FieldContext( + category = CATEGORY_POLICIES, + doc = "Enable consistent hashing for selecting the active consumer in partitioned " + + "topics with Failover subscription type. " + + "For non-partitioned topics, consistent hashing is used by default." +) +private boolean activeConsumerFailoverConsistentHashing = false; +``` + + +# Backward & Forward Compatibility +The default value is false to keep original behavior. From 49aa3080d422994baa036ed0b743a2fa18a6d530 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 21 Nov 2024 10:59:56 +0800 Subject: [PATCH 144/327] [improve][broker] PIP-392: Add configuration to enable consistent hashing to select active consumer for partitioned topic (#23584) --- conf/broker.conf | 4 +++ conf/standalone.conf | 4 +++ .../terraform-ansible/templates/broker.conf | 4 +++ .../pulsar/broker/ServiceConfiguration.java | 7 ++++ ...bstractDispatcherSingleActiveConsumer.java | 2 +- .../client/impl/TopicsConsumerImplTest.java | 33 +++++++++++++++++++ 6 files changed, 53 insertions(+), 1 deletion(-) diff --git a/conf/broker.conf b/conf/broker.conf index e745fcb2b0a8f..af335c141534f 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -243,6 +243,10 @@ messageExpiryCheckIntervalInMinutes=5 # How long to delay rewinding cursor and dispatching messages when active consumer is changed activeConsumerFailoverDelayTimeMillis=1000 +# Enable consistent hashing for selecting the active consumer in partitioned topics with Failover subscription type. +# For non-partitioned topics, consistent hashing is used by default. +activeConsumerFailoverConsistentHashing=false + # How long to delete inactive subscriptions from last consuming # When it is 0, inactive subscriptions are not deleted automatically subscriptionExpirationTimeMinutes=0 diff --git a/conf/standalone.conf b/conf/standalone.conf index 535800a43f3e0..90cf3b57ff941 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -152,6 +152,10 @@ maxMessageSizeCheckIntervalInSeconds=60 # How long to delay rewinding cursor and dispatching messages when active consumer is changed activeConsumerFailoverDelayTimeMillis=1000 +# Enable consistent hashing for selecting the active consumer in partitioned topics with Failover subscription type. +# For non-partitioned topics, consistent hashing is used by default. +activeConsumerFailoverConsistentHashing=false + # How long to delete inactive subscriptions from last consuming # When it is 0, inactive subscriptions are not deleted automatically subscriptionExpirationTimeMinutes=0 diff --git a/deployment/terraform-ansible/templates/broker.conf b/deployment/terraform-ansible/templates/broker.conf index ff3677174024c..bae55cb69f1ee 100644 --- a/deployment/terraform-ansible/templates/broker.conf +++ b/deployment/terraform-ansible/templates/broker.conf @@ -148,6 +148,10 @@ messageExpiryCheckIntervalInMinutes=5 # How long to delay rewinding cursor and dispatching messages when active consumer is changed activeConsumerFailoverDelayTimeMillis=1000 +# Enable consistent hashing for selecting the active consumer in partitioned topics with Failover subscription type. +# For non-partitioned topics, consistent hashing is used by default. +activeConsumerFailoverConsistentHashing=false + # How long to delete inactive subscriptions from last consuming # When it is 0, inactive subscriptions are not deleted automatically subscriptionExpirationTimeMinutes=0 diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 19e9ff625cada..8b5a4ef270b0e 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -760,6 +760,13 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "How long to delay rewinding cursor and dispatching messages when active consumer is changed" ) private int activeConsumerFailoverDelayTimeMillis = 1000; + @FieldContext( + category = CATEGORY_POLICIES, + doc = "Enable consistent hashing for selecting the active consumer in partitioned " + + "topics with Failover subscription type." + + "For non-partitioned topics, consistent hashing is used by default." + ) + private boolean activeConsumerFailoverConsistentHashing = false; @FieldContext( category = CATEGORY_POLICIES, doc = "Maximum time to spend while scanning a subscription to calculate the accurate backlog" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java index 9980b6ae97c6a..baca6bf078cf0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java @@ -126,7 +126,7 @@ protected boolean pickAndScheduleActiveConsumer() { } } } - int index = partitionIndex >= 0 + int index = partitionIndex >= 0 && !serviceConfig.isActiveConsumerFailoverConsistentHashing() ? partitionIndex % consumersSize : peekConsumerIndexFromHashRing(makeHashRing(consumersSize)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java index 83cb5f2a4400b..1d5ac75962524 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java @@ -1380,6 +1380,39 @@ public void testTopicsDistribution() throws Exception { } } + @Test + public void testPartitionedTopicDistribution() throws Exception { + this.conf.setActiveConsumerFailoverConsistentHashing(true); + final String topic = "partitioned-topics-distribution"; + final int topicCount = 100; + final int consumers = 10; + + for (int i = 0; i < topicCount; i++) { + admin.topics().createPartitionedTopic(topic + "-" + i, 1); + } + + CustomizedConsumerEventListener eventListener = new CustomizedConsumerEventListener(); + + List> consumerList = new ArrayList<>(consumers); + for (int i = 0; i < consumers; i++) { + consumerList.add(pulsarClient.newConsumer() + .topics(IntStream.range(0, topicCount).mapToObj(j -> topic + "-" + j).toList()) + .subscriptionType(SubscriptionType.Failover) + .subscriptionName("my-sub") + .consumerName("consumer-" + i) + .consumerEventListener(eventListener) + .subscribe()); + } + + log.info("Topics are distributed to consumers as {}", eventListener.getActiveConsumers()); + Map assigned = new HashMap<>(); + eventListener.getActiveConsumers().forEach((k, v) -> assigned.compute(v, (t, c) -> c == null ? 1 : ++ c)); + assertEquals(assigned.size(), consumers); + for (Consumer consumer : consumerList) { + consumer.close(); + } + } + private static class CustomizedConsumerEventListener implements ConsumerEventListener { private final Map activeConsumers = new HashMap<>(); From 949750fc080f3bee964dc839d16c9e215465d93a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 21 Nov 2024 10:13:23 +0200 Subject: [PATCH 145/327] [improve] Improve logic for enabling Netty leak detection (#23613) --- .../allocator/PulsarByteBufAllocator.java | 29 ++++++- .../allocator/PulsarByteBufAllocatorTest.java | 76 +++++++++++++++++++ 2 files changed, 103 insertions(+), 2 deletions(-) create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/allocator/PulsarByteBufAllocatorTest.java diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/allocator/PulsarByteBufAllocator.java b/pulsar-common/src/main/java/org/apache/pulsar/common/allocator/PulsarByteBufAllocator.java index ac12bb2df124d..4ad0732a62d74 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/allocator/PulsarByteBufAllocator.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/allocator/PulsarByteBufAllocator.java @@ -21,9 +21,13 @@ import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.PooledByteBufAllocator; +import java.util.Arrays; +import java.util.Comparator; import java.util.List; +import java.util.Objects; import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.Consumer; +import java.util.function.Function; import lombok.experimental.UtilityClass; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.allocator.ByteBufAllocatorBuilder; @@ -44,6 +48,13 @@ public class PulsarByteBufAllocator { public static final String PULSAR_ALLOCATOR_LEAK_DETECTION = "pulsar.allocator.leak_detection"; public static final String PULSAR_ALLOCATOR_OUT_OF_MEMORY_POLICY = "pulsar.allocator.out_of_memory_policy"; + // the highest level of leak detection policy will be used when it is set by any of the following property names + private static final String[] LEAK_DETECTION_PROPERTY_NAMES = { + PULSAR_ALLOCATOR_LEAK_DETECTION, + "io.netty.leakDetection.level", // io.netty.util.ResourceLeakDetector.PROP_LEVEL + "io.netty.leakDetectionLevel" // io.netty.util.ResourceLeakDetector.PROP_LEVEL_OLD + }; + public static final ByteBufAllocator DEFAULT; private static final List> LISTENERS = new CopyOnWriteArrayList<>(); @@ -64,8 +75,7 @@ static ByteBufAllocator createByteBufAllocator() { final OutOfMemoryPolicy outOfMemoryPolicy = OutOfMemoryPolicy.valueOf( System.getProperty(PULSAR_ALLOCATOR_OUT_OF_MEMORY_POLICY, "FallbackToHeap")); - final LeakDetectionPolicy leakDetectionPolicy = LeakDetectionPolicy - .valueOf(System.getProperty(PULSAR_ALLOCATOR_LEAK_DETECTION, "Disabled")); + final LeakDetectionPolicy leakDetectionPolicy = resolveLeakDetectionPolicyWithHighestLevel(System::getProperty); if (log.isDebugEnabled()) { log.debug("Is Pooled: {} -- Exit on OOM: {}", isPooled, isExitOnOutOfMemory); } @@ -98,4 +108,19 @@ static ByteBufAllocator createByteBufAllocator() { return builder.build(); } + + /** + * Resolve the leak detection policy. The value is resolved from the system properties in + * the order of LEAK_DETECTION_PROPERTY_NAMES. + * @return parsed leak detection policy + */ + @VisibleForTesting + static LeakDetectionPolicy resolveLeakDetectionPolicyWithHighestLevel(Function propertyResolver) { + return Arrays.stream(LEAK_DETECTION_PROPERTY_NAMES) + .map(propertyResolver) + .filter(Objects::nonNull) + .map(LeakDetectionPolicy::parseLevel) + .max(Comparator.comparingInt(Enum::ordinal)) + .orElse(LeakDetectionPolicy.Disabled); + } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/allocator/PulsarByteBufAllocatorTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/allocator/PulsarByteBufAllocatorTest.java new file mode 100644 index 0000000000000..4c69a1938b956 --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/allocator/PulsarByteBufAllocatorTest.java @@ -0,0 +1,76 @@ +/* + * 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.common.allocator; + +import static org.testng.Assert.assertEquals; +import java.util.Properties; +import org.apache.bookkeeper.common.allocator.LeakDetectionPolicy; +import org.testng.annotations.Test; + +public class PulsarByteBufAllocatorTest { + + @Test + public void testResolveLeakDetectionPolicyWithHighestLevel() { + Properties properties = new Properties(); + properties.setProperty("io.netty.leakDetectionLevel", "paranoid"); + properties.setProperty("io.netty.leakDetection.level", "advanced"); + properties.setProperty("pulsar.allocator.leak_detection", "simple"); + assertEquals(PulsarByteBufAllocator.resolveLeakDetectionPolicyWithHighestLevel(properties::getProperty), + LeakDetectionPolicy.Paranoid); + + properties.setProperty("io.netty.leakDetectionLevel", "advanced"); + properties.setProperty("io.netty.leakDetection.level", "simple"); + properties.setProperty("pulsar.allocator.leak_detection", "paranoid"); + assertEquals(PulsarByteBufAllocator.resolveLeakDetectionPolicyWithHighestLevel(properties::getProperty), + LeakDetectionPolicy.Paranoid); + + properties.setProperty("io.netty.leakDetectionLevel", "simple"); + properties.setProperty("io.netty.leakDetection.level", "paranoid"); + properties.setProperty("pulsar.allocator.leak_detection", "advanced"); + assertEquals(PulsarByteBufAllocator.resolveLeakDetectionPolicyWithHighestLevel(properties::getProperty), + LeakDetectionPolicy.Paranoid); + + properties.setProperty("io.netty.leakDetectionLevel", "disabled"); + properties.setProperty("io.netty.leakDetection.level", "simple"); + properties.setProperty("pulsar.allocator.leak_detection", "disabled"); + assertEquals(PulsarByteBufAllocator.resolveLeakDetectionPolicyWithHighestLevel(properties::getProperty), + LeakDetectionPolicy.Simple); + + properties.setProperty("io.netty.leakDetectionLevel", "invalid"); + properties.setProperty("io.netty.leakDetection.level", "invalid"); + properties.setProperty("pulsar.allocator.leak_detection", "invalid"); + assertEquals(PulsarByteBufAllocator.resolveLeakDetectionPolicyWithHighestLevel(properties::getProperty), + LeakDetectionPolicy.Disabled); + + properties.clear(); + properties.setProperty("pulsar.allocator.leak_detection", "Paranoid"); + assertEquals(PulsarByteBufAllocator.resolveLeakDetectionPolicyWithHighestLevel(properties::getProperty), + LeakDetectionPolicy.Paranoid); + + properties.clear(); + properties.setProperty("io.netty.leakDetectionLevel", "Advanced"); + assertEquals(PulsarByteBufAllocator.resolveLeakDetectionPolicyWithHighestLevel(properties::getProperty), + LeakDetectionPolicy.Advanced); + + properties.clear(); + properties.setProperty("io.netty.leakDetection.level", "Simple"); + assertEquals(PulsarByteBufAllocator.resolveLeakDetectionPolicyWithHighestLevel(properties::getProperty), + LeakDetectionPolicy.Simple); + } +} \ No newline at end of file From 024ff7574b55104b2460c7969eb127577bfb54dc Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Fri, 22 Nov 2024 09:05:01 +0800 Subject: [PATCH 146/327] [fix][fn] ack messages for window function when its result is null (#23618) --- .../windowing/WindowFunctionExecutor.java | 7 +++++++ .../functions/PulsarFunctionsTest.java | 20 ++++++++++++++++++- 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/windowing/WindowFunctionExecutor.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/windowing/WindowFunctionExecutor.java index c6ca4e65d33c0..1e492d74aa605 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/windowing/WindowFunctionExecutor.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/windowing/WindowFunctionExecutor.java @@ -238,6 +238,13 @@ private void processWindow(Context context, List> tuples, List record : tuples) { + record.ack(); + } + } } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java index b78a832f60933..694dcba5eaf61 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.fasterxml.jackson.databind.JsonNode; @@ -322,7 +323,17 @@ protected void testWindowFunction(String type, String[] expectedResults) throws .enableBatching(false) .create(); - for (int i = 0; i < NUM_OF_MESSAGES; i++) { + // send 3 messages first, and it won't trigger the window and so these 3 messages will not be acked + for (int i = 0; i < 3; i++) { + producer.send(String.format("%d", i).getBytes()); + } + TopicStats stats = pulsarAdmin.topics().getStats(inputTopicName, true); + SubscriptionStats subStats = stats.getSubscriptions().get("public/default/" + functionName); + assertNotNull(subStats); + assertEquals(3, subStats.getMsgBacklog()); + assertEquals(3, subStats.getUnackedMessages()); + + for (int i = 3; i < NUM_OF_MESSAGES; i++) { producer.send(String.format("%d", i).getBytes()); } @@ -348,6 +359,13 @@ protected void testWindowFunction(String type, String[] expectedResults) throws // in case last commit is not updated assertThat(i).isGreaterThanOrEqualTo(expectedResults.length - 1); + // test that all messages are acked + stats = pulsarAdmin.topics().getStats(inputTopicName, true); + subStats = stats.getSubscriptions().get("public/default/" + functionName); + assertNotNull(subStats); + assertEquals(0, subStats.getMsgBacklog()); + assertEquals(0, subStats.getUnackedMessages()); + deleteFunction(functionName); getFunctionInfoNotFound(functionName); From 708c5cc0c5f86d6c6bbdb438067122074f4de994 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Fri, 22 Nov 2024 09:51:02 +0800 Subject: [PATCH 147/327] [fix][client] fix incomingMessageSize and client memory usage is negative (#23624) Co-authored-by: fanjianye --- .../api/SimpleProducerConsumerTest.java | 56 +++++++++++++++++ .../impl/AutoScaledReceiverQueueSizeTest.java | 62 +++++++++++++++++++ .../pulsar/client/impl/ConsumerBase.java | 5 ++ .../pulsar/client/impl/ConsumerImpl.java | 2 + 4 files changed, 125 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 78d28e4b22834..9e35b4f262e61 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -4252,6 +4252,62 @@ public void testIncomingMessageSize(boolean isPartitioned) throws Exception { }); } + @Test(timeOut = 100000) + public void testNegativeIncomingMessageSize() throws Exception { + final String topicName = "persistent://my-property/my-ns/testIncomingMessageSize-" + + UUID.randomUUID().toString(); + final String subName = "my-sub"; + + admin.topics().createPartitionedTopic(topicName, 3); + + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topicName) + .enableBatching(false) + .create(); + + final int messages = 1000; + List> messageIds = new ArrayList<>(messages); + for (int i = 0; i < messages; i++) { + messageIds.add(producer.newMessage().key(i + "").value(("Message-" + i).getBytes()).sendAsync()); + } + FutureUtil.waitForAll(messageIds).get(); + + @Cleanup + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + + Awaitility.await().untilAsserted(() -> { + long size = ((ConsumerBase) consumer).getIncomingMessageSize(); + log.info("Check the incoming message size should greater that 0, current size is {}", size); + Assert.assertTrue(size > 0); + }); + + + for (int i = 0; i < messages; i++) { + consumer.receive(); + } + + + Awaitility.await().untilAsserted(() -> { + long size = ((ConsumerBase) consumer).getIncomingMessageSize(); + log.info("Check the incoming message size should be 0, current size is {}", size); + Assert.assertEquals(size, 0); + }); + + + MultiTopicsConsumerImpl multiTopicsConsumer = (MultiTopicsConsumerImpl) consumer; + List> list = multiTopicsConsumer.getConsumers(); + for (ConsumerImpl subConsumer : list) { + long size = subConsumer.getIncomingMessageSize(); + log.info("Check the sub consumer incoming message size should be 0, current size is {}", size); + Assert.assertEquals(size, 0); + } + } @Data @EqualsAndHashCode diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoScaledReceiverQueueSizeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoScaledReceiverQueueSizeTest.java index 858e43e84656f..5359158bf7214 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoScaledReceiverQueueSizeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoScaledReceiverQueueSizeTest.java @@ -20,14 +20,22 @@ import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.BatchReceivePolicy; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -257,4 +265,58 @@ public void testMultiConsumerImplBatchReceive() throws PulsarClientException, Pu Awaitility.await().until(() -> consumer.getCurrentReceiverQueueSize() == currentSize * 2); log.info("getCurrentReceiverQueueSize={}", consumer.getCurrentReceiverQueueSize()); } + + @Test + public void testNegativeClientMemory() throws Exception { + final String topicName = "persistent://public/default/testMemory-" + + UUID.randomUUID().toString(); + final String subName = "my-sub"; + + admin.topics().createPartitionedTopic(topicName, 3); + + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topicName) + .enableBatching(false) + .create(); + + final int messages = 1000; + List> messageIds = new ArrayList<>(messages); + for (int i = 0; i < messages; i++) { + messageIds.add(producer.newMessage().key(i + "").value(("Message-" + i).getBytes()).sendAsync()); + } + FutureUtil.waitForAll(messageIds).get(); + + + @Cleanup + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .autoScaledReceiverQueueSizeEnabled(true) + .subscribe(); + + + Awaitility.await().untilAsserted(() -> { + long size = ((ConsumerBase) consumer).getIncomingMessageSize(); + log.info("Check the incoming message size should greater that 0, current size is {}", size); + Assert.assertTrue(size > 0); + }); + + + for (int i = 0; i < messages; i++) { + consumer.receive(); + } + + Awaitility.await().untilAsserted(() -> { + long size = ((ConsumerBase) consumer).getIncomingMessageSize(); + log.info("Check the incoming message size should be 0, current size is {}", size); + Assert.assertEquals(size, 0); + }); + + + MemoryLimitController controller = ((PulsarClientImpl)pulsarClient).getMemoryLimitController(); + Assert.assertEquals(controller.currentUsage(), 0); + Assert.assertEquals(controller.currentUsagePercent(), 0); + } } 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 0fc906b6e7a9e..8c10577bc8617 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 @@ -1232,6 +1232,11 @@ protected void decreaseIncomingMessageSize(final Message message) { getMemoryLimitController().ifPresent(limiter -> limiter.releaseMemory(message.size())); } + protected void increaseIncomingMessageSize(final Message message) { + INCOMING_MESSAGES_SIZE_UPDATER.addAndGet(this, message.size()); + getMemoryLimitController().ifPresent(limiter -> limiter.forceReserveMemory(message.size())); + } + public long getIncomingMessageSize() { return INCOMING_MESSAGES_SIZE_UPDATER.get(this); } 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 004adab56f529..ffdf4cfdc8b30 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 @@ -1668,6 +1668,8 @@ void notifyPendingReceivedCallback(final Message message, Exception exception return; } + // increase incomingMessageSize here because the size would be decreased in messageProcessed() next step + increaseIncomingMessageSize(message); // increase permits for available message-queue messageProcessed(message); // call interceptor and complete received callback From d33cc20761b97d103d52ce7e24638edcd43a2a1e Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 22 Nov 2024 14:10:05 +0800 Subject: [PATCH 148/327] [improve][broker] Reduce memory occupation of the delayed message queue (#23611) --- .../InMemoryDelayedDeliveryTracker.java | 114 +++++++++++++++--- .../delayed/InMemoryDeliveryTrackerTest.java | 8 +- 2 files changed, 100 insertions(+), 22 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index bdc6e4c814e33..5796fcbd78550 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -20,6 +20,12 @@ import com.google.common.annotations.VisibleForTesting; import io.netty.util.Timer; +import it.unimi.dsi.fastutil.longs.Long2ObjectAVLTreeMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectSortedMap; +import it.unimi.dsi.fastutil.longs.LongOpenHashSet; +import it.unimi.dsi.fastutil.longs.LongSet; import java.time.Clock; import java.util.NavigableSet; import java.util.TreeSet; @@ -29,12 +35,15 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; -import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; +import org.roaringbitmap.longlong.Roaring64Bitmap; @Slf4j public class InMemoryDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker { - protected final TripleLongPriorityQueue priorityQueue = new TripleLongPriorityQueue(); + // timestamp -> ledgerId -> entryId + // AVL tree -> OpenHashMap -> RoaringBitmap + protected final Long2ObjectSortedMap> + delayedMessageMap = new Long2ObjectAVLTreeMap<>(); // If we detect that all messages have fixed delay time, such that the delivery is // always going to be in FIFO order, then we can avoid pulling all the messages in @@ -52,6 +61,9 @@ public class InMemoryDelayedDeliveryTracker extends AbstractDelayedDeliveryTrack // Track whether we have seen all messages with fixed delay so far. private boolean messagesHaveFixedDelay = true; + // The bit count to trim to reduce memory occupation. + private final int timestampPrecisionBitCnt; + InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, @@ -66,6 +78,35 @@ public InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsum long fixedDelayDetectionLookahead) { super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); this.fixedDelayDetectionLookahead = fixedDelayDetectionLookahead; + this.timestampPrecisionBitCnt = calculateTimestampPrecisionBitCnt(tickTimeMillis); + } + + /** + * The tick time is used to determine the precision of the delivery time. As the redelivery time + * is not accurate, we can bucket the delivery time and group multiple message ids into the same + * bucket to reduce the memory usage. THe default value is 1 second, which means we accept 1 second + * deviation for the delivery time, so that we can trim the lower 9 bits of the delivery time, because + * 2**9ms = 512ms < 1s, 2**10ms = 1024ms > 1s. + * @param tickTimeMillis + * @return + */ + private static int calculateTimestampPrecisionBitCnt(long tickTimeMillis) { + int bitCnt = 0; + while (tickTimeMillis > 0) { + tickTimeMillis >>= 1; + bitCnt++; + } + return bitCnt > 0 ? bitCnt - 1 : 0; + } + + /** + * trim the lower bits of the timestamp to reduce the memory usage. + * @param timestamp + * @param bits + * @return + */ + private static long trimLowerBit(long timestamp, int bits) { + return timestamp & (-1L << bits); } @Override @@ -80,7 +121,10 @@ public boolean addMessage(long ledgerId, long entryId, long deliverAt) { deliverAt - clock.millis()); } - priorityQueue.add(deliverAt, ledgerId, entryId); + long timestamp = trimLowerBit(deliverAt, timestampPrecisionBitCnt); + delayedMessageMap.computeIfAbsent(timestamp, k -> new Long2ObjectOpenHashMap<>()) + .computeIfAbsent(ledgerId, k -> new Roaring64Bitmap()) + .add(entryId); updateTimer(); checkAndUpdateHighest(deliverAt); @@ -105,7 +149,8 @@ private void checkAndUpdateHighest(long deliverAt) { */ @Override public boolean hasMessageAvailable() { - boolean hasMessageAvailable = !priorityQueue.isEmpty() && priorityQueue.peekN1() <= getCutoffTime(); + boolean hasMessageAvailable = !delayedMessageMap.isEmpty() + && delayedMessageMap.firstLongKey() <= getCutoffTime(); if (!hasMessageAvailable) { updateTimer(); } @@ -121,25 +166,49 @@ public NavigableSet getScheduledMessages(int maxMessages) { NavigableSet positions = new TreeSet<>(); long cutoffTime = getCutoffTime(); - while (n > 0 && !priorityQueue.isEmpty()) { - long timestamp = priorityQueue.peekN1(); + while (n > 0 && !delayedMessageMap.isEmpty()) { + long timestamp = delayedMessageMap.firstLongKey(); if (timestamp > cutoffTime) { break; } - long ledgerId = priorityQueue.peekN2(); - long entryId = priorityQueue.peekN3(); - positions.add(PositionFactory.create(ledgerId, entryId)); - - priorityQueue.pop(); - --n; + LongSet ledgerIdToDelete = new LongOpenHashSet(); + Long2ObjectMap ledgerMap = delayedMessageMap.get(timestamp); + for (Long2ObjectMap.Entry ledgerEntry : ledgerMap.long2ObjectEntrySet()) { + long ledgerId = ledgerEntry.getLongKey(); + Roaring64Bitmap entryIds = ledgerEntry.getValue(); + int cardinality = (int) entryIds.getLongCardinality(); + if (cardinality <= n) { + entryIds.forEach(entryId -> { + positions.add(PositionFactory.create(ledgerId, entryId)); + }); + n -= cardinality; + ledgerIdToDelete.add(ledgerId); + } else { + long[] entryIdsArray = entryIds.toArray(); + for (int i = 0; i < n; i++) { + positions.add(PositionFactory.create(ledgerId, entryIdsArray[i])); + entryIds.removeLong(entryIdsArray[i]); + } + n = 0; + } + if (n <= 0) { + break; + } + } + for (long ledgerId : ledgerIdToDelete) { + ledgerMap.remove(ledgerId); + } + if (ledgerMap.isEmpty()) { + delayedMessageMap.remove(timestamp); + } } if (log.isDebugEnabled()) { log.debug("[{}] Get scheduled messages - found {}", dispatcher.getName(), positions.size()); } - if (priorityQueue.isEmpty()) { + if (delayedMessageMap.isEmpty()) { // Reset to initial state highestDeliveryTimeTracked = 0; messagesHaveFixedDelay = true; @@ -151,24 +220,33 @@ public NavigableSet getScheduledMessages(int maxMessages) { @Override public CompletableFuture clear() { - this.priorityQueue.clear(); + this.delayedMessageMap.clear(); return CompletableFuture.completedFuture(null); } @Override public long getNumberOfDelayedMessages() { - return priorityQueue.size(); + return delayedMessageMap.values().stream().mapToLong( + ledgerMap -> ledgerMap.values().stream().mapToLong( + Roaring64Bitmap::getLongCardinality).sum()).sum(); } + /** + * This method rely on Roaring64Bitmap::getLongSizeInBytes to calculate the memory usage of the buffer. + * The memory usage of the buffer is not accurate, because Roaring64Bitmap::getLongSizeInBytes will + * overestimate the memory usage of the buffer a lot. + * @return the memory usage of the buffer + */ @Override public long getBufferMemoryUsage() { - return priorityQueue.bytesCapacity(); + return delayedMessageMap.values().stream().mapToLong( + ledgerMap -> ledgerMap.values().stream().mapToLong( + Roaring64Bitmap::getLongSizeInBytes).sum()).sum(); } @Override public void close() { super.close(); - priorityQueue.close(); } @Override @@ -181,6 +259,6 @@ && getNumberOfDelayedMessages() >= fixedDelayDetectionLookahead } protected long nextDeliveryTime() { - return priorityQueue.peekN1(); + return delayedMessageMap.firstLongKey(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java index ff7763927d888..dc6f623c82b57 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java @@ -92,7 +92,7 @@ public Object[][] provider(Method method) throws Exception { false, 0) }}; case "testAddMessageWithStrictDelay" -> new Object[][]{{ - new InMemoryDelayedDeliveryTracker(dispatcher, timer, 100, clock, + new InMemoryDelayedDeliveryTracker(dispatcher, timer, 1, clock, true, 0) }}; case "testAddMessageWithDeliverAtTimeAfterNowBeforeTickTimeFrequencyWithStrict" -> new Object[][]{{ @@ -100,7 +100,7 @@ public Object[][] provider(Method method) throws Exception { true, 0) }}; case "testAddMessageWithDeliverAtTimeAfterNowAfterTickTimeFrequencyWithStrict" -> new Object[][]{{ - new InMemoryDelayedDeliveryTracker(dispatcher, timer, 100000, clock, + new InMemoryDelayedDeliveryTracker(dispatcher, timer, 1, clock, true, 0) }}; case "testAddMessageWithDeliverAtTimeAfterFullTickTimeWithStrict" -> new Object[][]{{ @@ -108,7 +108,7 @@ public Object[][] provider(Method method) throws Exception { true, 0) }}; case "testWithFixedDelays", "testWithMixedDelays","testWithNoDelays" -> new Object[][]{{ - new InMemoryDelayedDeliveryTracker(dispatcher, timer, 500, clock, + new InMemoryDelayedDeliveryTracker(dispatcher, timer, 8, clock, true, 100) }}; default -> new Object[][]{{ @@ -230,7 +230,7 @@ public void run(Timeout timeout) throws Exception { return; } try { - this.priorityQueue.peekN1(); + this.delayedMessageMap.firstLongKey(); } catch (Exception e) { e.printStackTrace(); exceptions[0] = e; From e8657e2b94951b0b98797a6e1d943113121b1e53 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Fri, 22 Nov 2024 08:49:07 -0800 Subject: [PATCH 149/327] [improve][broker] Skip unloading when bundle throughput is zero (ExtensibleLoadManagerImpl only) (#23626) --- .../extensions/models/TopKBundles.java | 6 +++++ .../extensions/scheduler/TransferShedder.java | 10 +++++++ .../extensions/models/TopKBundlesTest.java | 26 +++++++++++++++++++ .../scheduler/TransferShedderTest.java | 20 ++++++++++++++ 4 files changed, 62 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java index ec26521af41f5..9c6e963417813 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java @@ -72,6 +72,12 @@ public void update(Map bundleStats, int topk) { pulsar.getConfiguration().isLoadBalancerSheddingBundlesWithPoliciesEnabled(); for (var etr : bundleStats.entrySet()) { String bundle = etr.getKey(); + var stat = etr.getValue(); + + // skip zero traffic bundles + if (stat.msgThroughputIn + stat.msgThroughputOut == 0) { + continue; + } // TODO: do not filter system topic while shedding if (NamespaceService.isSystemServiceNamespace(NamespaceBundle.getBundleNamespace(bundle))) { continue; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedder.java index 7126ccb034196..72d671aa4ca8d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedder.java @@ -528,6 +528,13 @@ public Set findBundlesForUnloading(LoadManagerContext context, var bundleData = e.stats(); double maxBrokerBundleThroughput = bundleData.msgThroughputIn + bundleData.msgThroughputOut; + if (maxBrokerBundleThroughput == 0) { + if (debugMode) { + log.info(String.format(CANNOT_UNLOAD_BUNDLE_MSG + + " It has zero throughput.", bundle)); + } + continue; + } boolean swap = false; List minToMaxUnloads = new ArrayList<>(); double minBrokerBundleSwapThroughput = 0.0; @@ -549,6 +556,9 @@ public Set findBundlesForUnloading(LoadManagerContext context, var minBrokerBundleThroughput = minBrokerBundleData.stats().msgThroughputIn + minBrokerBundleData.stats().msgThroughputOut; + if (minBrokerBundleThroughput == 0) { + continue; + } var maxBrokerNewThroughputTmp = maxBrokerNewThroughput + minBrokerBundleThroughput; var minBrokerNewThroughputTmp = minBrokerNewThroughput - minBrokerBundleThroughput; if (maxBrokerNewThroughputTmp < maxBrokerThroughput diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundlesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundlesTest.java index 472d44df8906d..3445ab393be7a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundlesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundlesTest.java @@ -88,14 +88,17 @@ public void testTopBundlesLoadData() { var topKBundles = new TopKBundles(pulsar); NamespaceBundleStats stats1 = new NamespaceBundleStats(); stats1.msgRateIn = 100000; + stats1.msgThroughputOut = 10; bundleStats.put(bundle1, stats1); NamespaceBundleStats stats2 = new NamespaceBundleStats(); stats2.msgRateIn = 500; + stats2.msgThroughputOut = 10; bundleStats.put(bundle2, stats2); NamespaceBundleStats stats3 = new NamespaceBundleStats(); stats3.msgRateIn = 10000; + stats3.msgThroughputOut = 10; bundleStats.put(bundle3, stats3); NamespaceBundleStats stats4 = new NamespaceBundleStats(); @@ -118,10 +121,12 @@ public void testSystemNamespace() { var topKBundles = new TopKBundles(pulsar); NamespaceBundleStats stats1 = new NamespaceBundleStats(); stats1.msgRateIn = 500; + stats1.msgThroughputOut = 10; bundleStats.put("pulsar/system/0x00000000_0x0FFFFFFF", stats1); NamespaceBundleStats stats2 = new NamespaceBundleStats(); stats2.msgRateIn = 10000; + stats2.msgThroughputOut = 10; bundleStats.put(bundle1, stats2); topKBundles.update(bundleStats, 2); @@ -131,6 +136,21 @@ public void testSystemNamespace() { assertEquals(top0.bundleName(), bundle1); } + @Test + public void testZeroMsgThroughputBundleStats() { + Map bundleStats = new HashMap<>(); + var topKBundles = new TopKBundles(pulsar); + NamespaceBundleStats stats1 = new NamespaceBundleStats(); + bundleStats.put(bundle1, stats1); + + NamespaceBundleStats stats2 = new NamespaceBundleStats(); + bundleStats.put(bundle1, stats2); + + topKBundles.update(bundleStats, 2); + + assertEquals(topKBundles.getLoadData().getTopBundlesLoadData().size(), 0); + } + private void setAntiAffinityGroup() throws MetadataStoreException { LocalPolicies localPolicies = new LocalPolicies(null, null, "namespaceAntiAffinityGroup"); @@ -166,10 +186,12 @@ public void testIsolationPolicy() throws MetadataStoreException { var topKBundles = new TopKBundles(pulsar); NamespaceBundleStats stats1 = new NamespaceBundleStats(); stats1.msgRateIn = 500; + stats1.msgThroughputOut = 10; bundleStats.put(bundle1, stats1); NamespaceBundleStats stats2 = new NamespaceBundleStats(); stats2.msgRateIn = 10000; + stats2.msgThroughputOut = 10; bundleStats.put(bundle2, stats2); topKBundles.update(bundleStats, 2); @@ -188,10 +210,12 @@ public void testAntiAffinityGroupPolicy() throws MetadataStoreException { var topKBundles = new TopKBundles(pulsar); NamespaceBundleStats stats1 = new NamespaceBundleStats(); stats1.msgRateIn = 500; + stats1.msgThroughputOut = 10; bundleStats.put(bundle1, stats1); NamespaceBundleStats stats2 = new NamespaceBundleStats(); stats2.msgRateIn = 10000; + stats2.msgThroughputOut = 10; bundleStats.put(bundle2, stats2); topKBundles.update(bundleStats, 2); @@ -213,10 +237,12 @@ public void testLoadBalancerSheddingBundlesWithPoliciesEnabledConfig() throws Me var topKBundles = new TopKBundles(pulsar); NamespaceBundleStats stats1 = new NamespaceBundleStats(); stats1.msgRateIn = 500; + stats1.msgThroughputOut = 10; bundleStats.put(bundle1, stats1); NamespaceBundleStats stats2 = new NamespaceBundleStats(); stats2.msgRateIn = 10000; + stats2.msgThroughputOut = 10; bundleStats.put(bundle2, stats2); topKBundles.update(bundleStats, 2); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java index 48bef15b5f80a..5e20b196c9a5a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java @@ -919,6 +919,26 @@ public void testBundleThroughputLargerThanOffloadThreshold() { assertEquals(counter.getLoadStd(), setupLoadStd); } + @Test + public void testZeroBundleThroughput() { + UnloadCounter counter = new UnloadCounter(); + TransferShedder transferShedder = new TransferShedder(counter); + var ctx = setupContext(); + var topBundlesLoadDataStore = ctx.topBundleLoadDataStore(); + for (var e : topBundlesLoadDataStore.entrySet()) { + for (var stat : e.getValue().getTopBundlesLoadData()) { + stat.stats().msgThroughputOut = 0; + stat.stats().msgThroughputIn = 0; + + } + } + var res = transferShedder.findBundlesForUnloading(ctx, Map.of(), Map.of()); + assertTrue(res.isEmpty()); + assertEquals(counter.getBreakdownCounters().get(Skip).get(NoBundles).get(), 1); + assertEquals(counter.getLoadAvg(), setupLoadAvg); + assertEquals(counter.getLoadStd(), setupLoadStd); + } + @Test public void testTargetStdAfterTransfer() { From c6561e190dd52deeecd95e13af6c5937eb3f047e Mon Sep 17 00:00:00 2001 From: zhou zhuohan <843520313@qq.com> Date: Mon, 25 Nov 2024 10:39:29 +0800 Subject: [PATCH 150/327] [fix][client] Fix DLQ producer name conflicts when there are same name consumers (#23577) --- .../pulsar/broker/service/ServerCnxTest.java | 10 +-- .../client/api/DeadLetterTopicTest.java | 88 ++++++++++++++++++- .../pulsar/client/impl/ConsumerBase.java | 4 +- .../pulsar/client/impl/ConsumerImpl.java | 5 +- .../client/impl/MultiTopicsConsumerImpl.java | 6 +- .../util/{ConsumerName.java => NameUtil.java} | 2 +- 6 files changed, 99 insertions(+), 16 deletions(-) rename pulsar-client/src/main/java/org/apache/pulsar/client/util/{ConsumerName.java => NameUtil.java} (97%) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 9a85995ab771f..59e9847b75a33 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -108,7 +108,7 @@ import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.ProducerAccessMode; import org.apache.pulsar.client.api.transaction.TxnID; -import org.apache.pulsar.client.util.ConsumerName; +import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.api.proto.AuthMethod; import org.apache.pulsar.common.api.proto.BaseCommand; @@ -1084,8 +1084,8 @@ public void testHandleConsumerAfterClientChannelInactive() throws Exception { final long consumerId = 1; final MutableInt requestId = new MutableInt(1); final String sName = successSubName; - final String cName1 = ConsumerName.generateRandomName(); - final String cName2 = ConsumerName.generateRandomName(); + final String cName1 = NameUtil.generateRandomName(); + final String cName2 = NameUtil.generateRandomName(); resetChannel(); setChannelConnected(); @@ -1126,8 +1126,8 @@ public void test2ndSubFailedIfDisabledConCheck() final long consumerId = 1; final MutableInt requestId = new MutableInt(1); final String sName = successSubName; - final String cName1 = ConsumerName.generateRandomName(); - final String cName2 = ConsumerName.generateRandomName(); + final String cName1 = NameUtil.generateRandomName(); + final String cName2 = NameUtil.generateRandomName(); // Disabled connection check. pulsar.getConfig().setConnectionLivenessCheckTimeoutMillis(-1); resetChannel(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java index 7d34ce33e0a7f..e46fddeacc117 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java @@ -36,6 +36,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; import lombok.Cleanup; import lombok.Data; import org.apache.avro.reflect.Nullable; @@ -261,7 +262,12 @@ public void testDeadLetterTopicWithProducerName() throws Exception { final String topic = "persistent://my-property/my-ns/dead-letter-topic"; final String subscription = "my-subscription"; final String consumerName = "my-consumer"; - String deadLetterProducerName = String.format("%s-%s-%s-DLQ", topic, subscription, consumerName); + Pattern deadLetterProducerNamePattern = + Pattern.compile("^persistent://my-property/my-ns/dead-letter-topic" + + "-my-subscription" + + "-my-consumer" + + "-[a-zA-Z0-9]{5}" + + "-DLQ$"); final int maxRedeliveryCount = 1; @@ -308,8 +314,9 @@ public void testDeadLetterTopicWithProducerName() throws Exception { int totalInDeadLetter = 0; do { Message message = deadLetterConsumer.receive(); - assertEquals(message.getProducerName(), deadLetterProducerName); - log.info("dead letter consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + assertTrue(deadLetterProducerNamePattern.matcher(message.getProducerName()).matches()); + log.info("dead letter consumer received message : {} {}, dead letter producer name : {}", + message.getMessageId(), new String(message.getData()), message.getProducerName()); deadLetterConsumer.acknowledge(message); totalInDeadLetter++; } while (totalInDeadLetter < sendMessages); @@ -318,6 +325,81 @@ public void testDeadLetterTopicWithProducerName() throws Exception { consumer.close(); } + + @Test(timeOut = 30000) + public void testMultipleSameNameConsumersToDeadLetterTopic() throws Exception { + final String topic = "persistent://my-property/my-ns/same-name-consumers-dead-letter-topic"; + final int maxRedeliveryCount = 1; + final int messageCount = 10; + final int consumerCount = 3; + + //1 start 3 parallel consumers + List> consumers = new ArrayList<>(); + final AtomicInteger totalReceived = new AtomicInteger(0); + @Cleanup("shutdownNow") + ExecutorService executor = Executors.newFixedThreadPool(consumerCount); + for (int i = 0; i < consumerCount; i++) { + executor.execute(() -> { + try { + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("my-subscription-DuplicatedMessage") + .subscriptionType(SubscriptionType.Shared) + .consumerName("my-consumer") + .ackTimeout(1001, TimeUnit.MILLISECONDS) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount) + .deadLetterTopic(topic + "-DLQ").build()) + .negativeAckRedeliveryDelay(1001, TimeUnit.MILLISECONDS) + .receiverQueueSize(100) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .messageListener((MessageListener) (consumer1, msg) -> { + totalReceived.getAndIncrement(); + //never ack + }) + .subscribe(); + consumers.add(consumer); + } catch (PulsarClientException e) { + fail(); + } + }); + } + + //2 send messages + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + for (int i = 0; i < messageCount; i++) { + producer.send(String.format("Message [%d]", i)); + } + + //3 start a DLQ consumer + Consumer deadLetterConsumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic + "-DLQ") + .subscriptionName("my-subscription-DuplicatedMessage-DLQ") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + int totalInDeadLetter = 0; + while (true) { + Message message = deadLetterConsumer.receive(10, TimeUnit.SECONDS); + if (message == null) { + break; + } + deadLetterConsumer.acknowledge(message); + totalInDeadLetter++; + } + + //4 The number of messages that consumers can consume should be equal to messageCount * (maxRedeliveryCount + 1) + assertEquals(totalReceived.get(), messageCount * (maxRedeliveryCount + 1)); + + //5 The message in DLQ should be equal to messageCount + assertEquals(totalInDeadLetter, messageCount); + + //6 clean up + producer.close(); + deadLetterConsumer.close(); + for (Consumer consumer : consumers) { + consumer.close(); + } + } + @DataProvider(name = "produceLargeMessages") public Object[][] produceLargeMessages() { return new Object[][] { { false }, { true } }; 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 8c10577bc8617..31aef2fd25abb 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 @@ -60,8 +60,8 @@ import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.transaction.TransactionImpl; -import org.apache.pulsar.client.util.ConsumerName; import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.client.util.NoOpLock; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandSubscribe; @@ -132,7 +132,7 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat this.maxReceiverQueueSize = receiverQueueSize; this.subscription = conf.getSubscriptionName(); this.conf = conf; - this.consumerName = conf.getConsumerName() == null ? ConsumerName.generateRandomName() : conf.getConsumerName(); + this.consumerName = conf.getConsumerName() == null ? NameUtil.generateRandomName() : conf.getConsumerName(); this.subscribeFuture = subscribeFuture; this.listener = conf.getMessageListener(); this.consumerEventListener = conf.getConsumerEventListener(); 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 ffdf4cfdc8b30..d46f4af1be748 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 @@ -98,6 +98,7 @@ import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.client.util.RetryMessageUtil; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.EncryptionContext; @@ -2265,8 +2266,8 @@ private void initDeadLetterProducerIfNeeded() { ((ProducerBuilderImpl) client.newProducer(Schema.AUTO_PRODUCE_BYTES(schema))) .initialSubscriptionName(this.deadLetterPolicy.getInitialSubscriptionName()) .topic(this.deadLetterPolicy.getDeadLetterTopic()) - .producerName(String.format("%s-%s-%s-DLQ", this.topicName, this.subscription, - this.consumerName)) + .producerName(String.format("%s-%s-%s-%s-DLQ", this.topicName, this.subscription, + this.consumerName, NameUtil.generateRandomName())) .blockIfQueueFull(false) .enableBatching(false) .enableChunking(true) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 528a140b81c2c..dd6a304d9985c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -68,8 +68,8 @@ import org.apache.pulsar.client.api.TopicMessageId; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.transaction.TransactionImpl; -import org.apache.pulsar.client.util.ConsumerName; import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.TopicName; @@ -113,7 +113,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { MultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { - this(client, DUMMY_TOPIC_NAME_PREFIX + ConsumerName.generateRandomName(), conf, executorProvider, + this(client, DUMMY_TOPIC_NAME_PREFIX + NameUtil.generateRandomName(), conf, executorProvider, subscribeFuture, schema, interceptors, createTopicIfDoesNotExist); } @@ -121,7 +121,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist, MessageId startMessageId, long startMessageRollbackDurationInSec) { - this(client, DUMMY_TOPIC_NAME_PREFIX + ConsumerName.generateRandomName(), conf, executorProvider, + this(client, DUMMY_TOPIC_NAME_PREFIX + NameUtil.generateRandomName(), conf, executorProvider, subscribeFuture, schema, interceptors, createTopicIfDoesNotExist, startMessageId, startMessageRollbackDurationInSec); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/util/ConsumerName.java b/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java similarity index 97% rename from pulsar-client/src/main/java/org/apache/pulsar/client/util/ConsumerName.java rename to pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java index 56b47ad44c394..4c416b6152cf8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/util/ConsumerName.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java @@ -21,7 +21,7 @@ import java.util.UUID; import org.apache.commons.codec.digest.DigestUtils; -public class ConsumerName { +public class NameUtil { public static String generateRandomName() { return DigestUtils.sha1Hex(UUID.randomUUID().toString()).substring(0, 5); } From 2126d4016e43a21c044f2dd4f021fe7103ec83b0 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 25 Nov 2024 17:00:06 +0200 Subject: [PATCH 151/327] [improve][test] Reduce OneWayReplicatorUsingGlobalZKTest.testRemoveCluster execution time (#23633) --- .../broker/service/OneWayReplicatorUsingGlobalZKTest.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java index ad877e8f947b7..9287b451c2cdd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java @@ -203,13 +203,15 @@ public void testRemoveCluster() throws Exception { // The topics under the namespace of the cluster-1 will be deleted. // Verify the result. admin1.namespaces().setNamespaceReplicationClusters(ns1, new HashSet<>(Arrays.asList(cluster2))); - Awaitility.await().atMost(Duration.ofSeconds(120)).untilAsserted(() -> { + Awaitility.await().atMost(Duration.ofSeconds(60)).ignoreExceptions().untilAsserted(() -> { Map>> tps = pulsar1.getBrokerService().getTopics(); assertFalse(tps.containsKey(topic)); assertFalse(tps.containsKey(topicChangeEvents)); - assertFalse(pulsar1.getNamespaceService().checkTopicExists(TopicName.get(topic)).join().isExists()); + assertFalse(pulsar1.getNamespaceService().checkTopicExists(TopicName.get(topic)) + .get(5, TimeUnit.SECONDS).isExists()); assertFalse(pulsar1.getNamespaceService() - .checkTopicExists(TopicName.get(topicChangeEvents)).join().isExists()); + .checkTopicExists(TopicName.get(topicChangeEvents)) + .get(5, TimeUnit.SECONDS).isExists()); }); // cleanup. From 7909d2dfdb4aad8053c133ce6a00d5dddf0b9db8 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Mon, 25 Nov 2024 23:02:23 +0800 Subject: [PATCH 152/327] [fix][fn] Align WindowContext with BaseContext (#23628) --- .../pulsar/functions/api/WindowContext.java | 82 +------------------ .../windowing/WindowContextImpl.java | 40 +++++++++ 2 files changed, 42 insertions(+), 80 deletions(-) diff --git a/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/WindowContext.java b/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/WindowContext.java index a733ddef6559a..c4814836f0a75 100644 --- a/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/WindowContext.java +++ b/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/WindowContext.java @@ -18,33 +18,16 @@ */ package org.apache.pulsar.functions.api; -import java.nio.ByteBuffer; import java.util.Collection; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; -import org.slf4j.Logger; @InterfaceAudience.Public @InterfaceStability.Stable -public interface WindowContext { - - /** - * The tenant this function belongs to. - * - * @return the tenant this function belongs to - */ - String getTenant(); - - /** - * The namespace this function belongs to. - * - * @return the namespace this function belongs to - */ - String getNamespace(); - +public interface WindowContext extends BaseContext { /** * The name of the function that we are executing. * @@ -59,20 +42,6 @@ public interface WindowContext { */ String getFunctionId(); - /** - * The id of the instance that invokes this function. - * - * @return the instance id - */ - int getInstanceId(); - - /** - * Get the number of instances that invoke this function. - * - * @return the number of instances that invoke this function. - */ - int getNumInstances(); - /** * The version of the function that we are executing. * @@ -101,45 +70,6 @@ public interface WindowContext { */ String getOutputSchemaType(); - /** - * The logger object that can be used to log in a function. - * - * @return the logger object - */ - Logger getLogger(); - - /** - * Increment the builtin distributed counter referred by key. - * - * @param key The name of the key - * @param amount The amount to be incremented - */ - void incrCounter(String key, long amount); - - /** - * Retrieve the counter value for the key. - * - * @param key name of the key - * @return the amount of the counter value for this key - */ - long getCounter(String key); - - /** - * Update the state value for the key. - * - * @param key name of the key - * @param value state value of the key - */ - void putState(String key, ByteBuffer value); - - /** - * Retrieve the state value for the key. - * - * @param key name of the key - * @return the state value for the key. - */ - ByteBuffer getState(String key); - /** * Get a map of all user-defined key/value configs for the function. * @@ -164,14 +94,6 @@ public interface WindowContext { */ Object getUserConfigValueOrDefault(String key, Object defaultValue); - /** - * Record a user defined metric. - * - * @param metricName The name of the metric - * @param value The value of the metric - */ - void recordMetric(String metricName, double value); - /** * Publish an object using serDe for serializing to the topic. * @@ -194,4 +116,4 @@ public interface WindowContext { * @return A future that completes when the framework is done publishing the message */ CompletableFuture publish(String topicName, T object); -} +} \ No newline at end of file diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/windowing/WindowContextImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/windowing/WindowContextImpl.java index a53b05252fd33..cb5710f0177ab 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/windowing/WindowContextImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/windowing/WindowContextImpl.java @@ -90,26 +90,61 @@ public Logger getLogger() { return this.context.getLogger(); } + @Override + public String getSecret(String secretName) { + return this.context.getSecret(secretName); + } + @Override public void incrCounter(String key, long amount) { this.context.incrCounter(key, amount); } + @Override + public CompletableFuture incrCounterAsync(String key, long amount) { + return this.context.incrCounterAsync(key, amount); + } + @Override public long getCounter(String key) { return this.context.getCounter(key); } + @Override + public CompletableFuture getCounterAsync(String key) { + return this.context.getCounterAsync(key); + } + @Override public void putState(String key, ByteBuffer value) { this.context.putState(key, value); } + @Override + public CompletableFuture putStateAsync(String key, ByteBuffer value) { + return this.context.putStateAsync(key, value); + } + @Override public ByteBuffer getState(String key) { return this.context.getState(key); } + @Override + public CompletableFuture getStateAsync(String key) { + return this.context.getStateAsync(key); + } + + @Override + public void deleteState(String key) { + this.context.deleteState(key); + } + + @Override + public CompletableFuture deleteStateAsync(String key) { + return this.context.deleteStateAsync(key); + } + @Override public Map getUserConfigMap() { return this.context.getUserConfigMap(); @@ -130,6 +165,11 @@ public void recordMetric(String metricName, double value) { this.context.recordMetric(metricName, value); } + @Override + public void fatal(Throwable t) { + this.context.fatal(t); + } + @Override public CompletableFuture publish(String topicName, T object) { return this.context.publish(topicName, object); From 81dfb439aeccb5d2d92b78905fdf400d7d183d18 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 26 Nov 2024 12:43:35 -0800 Subject: [PATCH 153/327] [fix][ws] Implement missing http header data functions in AuthenticationDataSubscription (#23638) --- .../AuthenticationDataSubscription.java | 15 ++++++ .../AuthenticationDataSubscriptionTest.java | 48 +++++++++++++++++++ 2 files changed, 63 insertions(+) create mode 100644 pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscriptionTest.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscription.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscription.java index 9a7324a6d077a..96b85989b6fc3 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscription.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscription.java @@ -72,6 +72,21 @@ public String getSubscription() { return subscription; } + @Override + public boolean hasDataFromHttp() { + return authData.hasDataFromHttp(); + } + + @Override + public String getHttpAuthType() { + return authData.getHttpAuthType(); + } + + @Override + public String getHttpHeader(String name) { + return authData.getHttpHeader(name); + } + public AuthenticationDataSource getAuthData() { return authData; } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscriptionTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscriptionTest.java new file mode 100644 index 0000000000000..8af7b87fd221b --- /dev/null +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscriptionTest.java @@ -0,0 +1,48 @@ +/* + * 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.authentication; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.testng.AssertJUnit.assertEquals; +import javax.servlet.http.HttpServletRequest; +import org.testng.annotations.Test; + +public class AuthenticationDataSubscriptionTest { + + AuthenticationDataSubscription target; + + @Test + public void testTargetFromAuthenticationDataHttp(){ + var req = mock(HttpServletRequest.class); + String headerName = "Authorization"; + String headerValue = "my-header"; + String authType = "my-authType"; + doReturn(headerValue).when(req).getHeader(eq(headerName)); + doReturn("localhost").when(req).getRemoteAddr(); + doReturn(4000).when(req).getRemotePort(); + doReturn(authType).when(req).getAuthType(); + AuthenticationDataSource authenticationDataSource = new AuthenticationDataHttp(req); + target = new AuthenticationDataSubscription(authenticationDataSource, "my-sub"); + assertEquals(headerValue, target.getHttpHeader(headerName)); + assertEquals(authType, target.getHttpAuthType()); + assertEquals(true, target.hasDataFromHttp()); + } +} From 3e108da4f136e9c0013c8030e496d00ea94bfd2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=90=A7=E6=98=93=E5=AE=A2?= Date: Wed, 27 Nov 2024 05:09:21 +0800 Subject: [PATCH 154/327] [fix][misc] Class conflict during jetcd-core-shaded shading process (#23641) --- jetcd-core-shaded/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index b0e6e99052599..0b79df8278f81 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -94,6 +94,10 @@ io.etcd:* io.vertx:* + + + org.apache.pulsar:jetcd-core-shaded + From 13f77aa5b09dc48cd1326096a3defc9394fedf67 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 26 Nov 2024 13:20:51 -0800 Subject: [PATCH 155/327] [fix][client] Initializing client-authentication using configured auth params (#23610) --- .../broker/service/BrokerServiceTest.java | 45 +++++++++++++++++++ .../pulsar/client/impl/ClientBuilderImpl.java | 4 ++ 2 files changed, 49 insertions(+) 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 e05bb836a3ce6..89727014be99e 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 @@ -103,6 +103,7 @@ import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.ClientBuilderImpl; import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarServiceNameResolver; @@ -1916,5 +1917,49 @@ public void close() { } } } + + @Test + public void testTlsWithAuthParams() throws Exception { + final String topicName = "persistent://prop/ns-abc/newTopic"; + final String subName = "newSub"; + Authentication auth; + + Set providers = new HashSet<>(); + providers.add("org.apache.pulsar.broker.authentication.AuthenticationProviderTls"); + + conf.setAuthenticationEnabled(true); + conf.setAuthenticationProviders(providers); + conf.setBrokerServicePortTls(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + conf.setTlsAllowInsecureConnection(false); + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + conf.setNumExecutorThreadPoolSize(5); + restartBroker(); + + String authParam = String.format("tlsCertFile:%s,tlsKeyFile:%s", getTlsFileForClient("admin.cert"), + getTlsFileForClient("admin.key-pk8")); + String authClassName = "org.apache.pulsar.client.impl.auth.AuthenticationTls"; + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setServiceUrl(brokerUrlTls.toString()); + conf.setAuthParams(authParam); + conf.setAuthPluginClassName(authClassName); + conf.setTlsAllowInsecureConnection(true); + + PulsarClient pulsarClient = null; + try { + pulsarClient = (new ClientBuilderImpl(conf)).build(); + + @Cleanup + Consumer consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subName) + .subscribe(); + } catch (Exception e) { + fail("should not fail"); + } finally { + pulsarClient.close(); + } + } + } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java index 6923218676743..7652d8a41994d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.apache.pulsar.client.api.SizeUnit; +import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils; import org.apache.pulsar.common.tls.InetAddressUtils; @@ -64,6 +65,9 @@ public PulsarClient build() throws PulsarClientException { "Cannot get service url from service url provider."); conf.setServiceUrl(conf.getServiceUrlProvider().getServiceUrl()); } + if (conf.getAuthentication() == null || conf.getAuthentication() == AuthenticationDisabled.INSTANCE) { + setAuthenticationFromPropsIfAvailable(conf); + } PulsarClient client = new PulsarClientImpl(conf); if (conf.getServiceUrlProvider() != null) { conf.getServiceUrlProvider().initialize(client); From dbfde02bd740d5a682575ee971e2eacc2b28b3d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Herv=C3=A9=20Boutemy?= Date: Wed, 27 Nov 2024 08:33:31 +0100 Subject: [PATCH 156/327] [cleanup][build] skip generating pom.xml.versionsBackup (#23639) --- src/set-project-version.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/set-project-version.sh b/src/set-project-version.sh index f3f01009b19ef..015b7f14cd924 100755 --- a/src/set-project-version.sh +++ b/src/set-project-version.sh @@ -36,9 +36,9 @@ pushd ${ROOT_DIR} # Get the current version OLD_VERSION=`python3 ${ROOT_DIR}/src/get-project-version.py` -mvn versions:set -DnewVersion=$NEW_VERSION -mvn versions:set -DnewVersion=$NEW_VERSION -pl buildtools -mvn versions:set -DnewVersion=$NEW_VERSION -pl pulsar-bom +mvn versions:set -DnewVersion=$NEW_VERSION -DgenerateBackupPoms=false +mvn versions:set -DnewVersion=$NEW_VERSION -DgenerateBackupPoms=false -pl buildtools +mvn versions:set -DnewVersion=$NEW_VERSION -DgenerateBackupPoms=false -pl pulsar-bom # Set terraform ansible deployment pulsar version sed -i -e "s/${OLD_VERSION}/${NEW_VERSION}/g" ${TERRAFORM_DIR}/deploy-pulsar.yaml From 6b694f6f5020e95a874f755ca176334e0d5e6c56 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Wed, 27 Nov 2024 21:47:18 +0800 Subject: [PATCH 157/327] [improve][broker] Reduce memory occupation of InMemoryRedeliveryTracker. (#23640) --- .../service/InMemoryRedeliveryTracker.java | 84 ++++++++++++++----- 1 file changed, 65 insertions(+), 19 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/InMemoryRedeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/InMemoryRedeliveryTracker.java index 12e28793557b3..669562055214c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/InMemoryRedeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/InMemoryRedeliveryTracker.java @@ -18,49 +18,95 @@ */ package org.apache.pulsar.broker.service; +import it.unimi.dsi.fastutil.longs.Long2IntMap; +import it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap; import java.util.List; +import java.util.concurrent.locks.StampedLock; import org.apache.bookkeeper.mledger.Position; -import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair; public class InMemoryRedeliveryTracker implements RedeliveryTracker { - - private ConcurrentLongLongPairHashMap trackerCache = ConcurrentLongLongPairHashMap.newBuilder() - .concurrencyLevel(1) - .expectedItems(256) - .autoShrink(true) - .build(); + // ledgerId -> entryId -> count + private Long2ObjectMap trackerCache = new Long2ObjectOpenHashMap<>(); + private final StampedLock rwLock = new StampedLock(); @Override public int incrementAndGetRedeliveryCount(Position position) { - Position positionImpl = position; - LongPair count = trackerCache.get(positionImpl.getLedgerId(), positionImpl.getEntryId()); - int newCount = (int) (count != null ? count.first + 1 : 1); - trackerCache.put(positionImpl.getLedgerId(), positionImpl.getEntryId(), newCount, 0L); + long stamp = rwLock.writeLock(); + int newCount; + try { + Long2IntMap entryMap = trackerCache.computeIfAbsent(position.getLedgerId(), + k -> new Long2IntOpenHashMap()); + newCount = entryMap.getOrDefault(position.getEntryId(), 0) + 1; + entryMap.put(position.getEntryId(), newCount); + } finally { + rwLock.unlockWrite(stamp); + } return newCount; } @Override public int getRedeliveryCount(long ledgerId, long entryId) { - LongPair count = trackerCache.get(ledgerId, entryId); - return (int) (count != null ? count.first : 0); + long stamp = rwLock.tryOptimisticRead(); + Long2IntMap entryMap = trackerCache.get(ledgerId); + int count = entryMap != null ? entryMap.get(entryId) : 0; + if (!rwLock.validate(stamp)) { + stamp = rwLock.readLock(); + try { + entryMap = trackerCache.get(ledgerId); + count = entryMap != null ? entryMap.get(entryId) : 0; + } finally { + rwLock.unlockRead(stamp); + } + } + return count; } @Override public void remove(Position position) { - Position positionImpl = position; - trackerCache.remove(positionImpl.getLedgerId(), positionImpl.getEntryId()); + long stamp = rwLock.writeLock(); + try { + Long2IntMap entryMap = trackerCache.get(position.getLedgerId()); + if (entryMap != null) { + entryMap.remove(position.getEntryId()); + if (entryMap.isEmpty()) { + trackerCache.remove(position.getLedgerId()); + } + } + } finally { + rwLock.unlockWrite(stamp); + } } @Override public void removeBatch(List positions) { - if (positions != null) { - positions.forEach(this::remove); + if (positions == null) { + return; + } + long stamp = rwLock.writeLock(); + try { + for (Position position : positions) { + Long2IntMap entryMap = trackerCache.get(position.getLedgerId()); + if (entryMap != null) { + entryMap.remove(position.getEntryId()); + if (entryMap.isEmpty()) { + trackerCache.remove(position.getLedgerId()); + } + } + } + } finally { + rwLock.unlockWrite(stamp); } } @Override public void clear() { - trackerCache.clear(); + long stamp = rwLock.writeLock(); + try { + trackerCache.clear(); + } finally { + rwLock.unlockWrite(stamp); + } } } From 2c36410754007b43601561937c95db8fb0edf354 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=90=A7=E6=98=93=E5=AE=A2?= Date: Thu, 28 Nov 2024 04:34:07 +0800 Subject: [PATCH 158/327] [improve][misc] Disable OTel by default when running the pulsar-perf tool (#23585) --- .../java/org/apache/pulsar/testclient/PerfClientUtils.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java index 6bf73e705d16c..1e2f6231c6ed9 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java @@ -21,6 +21,7 @@ import static org.apache.commons.lang3.StringUtils.isNotBlank; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import java.lang.management.ManagementFactory; +import java.util.Map; import java.util.Objects; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -79,6 +80,9 @@ public static ClientBuilder createClientBuilderFromArguments(PerformanceBaseArgu .maxLookupRequests(arguments.maxLookupRequest) .proxyServiceUrl(arguments.proxyServiceURL, arguments.proxyProtocol) .openTelemetry(AutoConfiguredOpenTelemetrySdk.builder() + .addPropertiesSupplier(() -> Map.of( + "otel.sdk.disabled", "true" + )) .build().getOpenTelemetrySdk()); if (isNotBlank(arguments.authPluginClassName)) { From e236d61c650bcaef271bc529ec9023edd8c04b01 Mon Sep 17 00:00:00 2001 From: Yuri Mizushima Date: Thu, 28 Nov 2024 05:35:08 +0900 Subject: [PATCH 159/327] [fix][client] Fixed an issue where a cert chain could not be used in TLS authentication (#23644) --- .../pulsar/common/util/KeyManagerProxy.java | 19 +++---- .../common/util/KeyManagerProxyTest.java | 51 +++++++++++++++++++ 2 files changed, 61 insertions(+), 9 deletions(-) create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/KeyManagerProxyTest.java diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/KeyManagerProxy.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/KeyManagerProxy.java index 171f5acd2bf75..6b6b0492b4525 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/KeyManagerProxy.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/KeyManagerProxy.java @@ -30,11 +30,14 @@ import java.security.Principal; import java.security.PrivateKey; import java.security.UnrecoverableKeyException; +import java.security.cert.Certificate; import java.security.cert.CertificateException; import java.security.cert.CertificateFactory; import java.security.cert.X509Certificate; +import java.util.List; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import javax.net.ssl.KeyManagerFactory; import javax.net.ssl.SSLEngine; import javax.net.ssl.X509ExtendedKeyManager; @@ -88,18 +91,16 @@ private void updateKeyManager() return; } - X509Certificate certificate; - PrivateKey privateKey = null; - KeyStore keyStore; - try (InputStream publicCertStream = new FileInputStream(certFile.getFileName()); - InputStream privateKeyStream = new FileInputStream(keyFile.getFileName())) { + final KeyStore keyStore; + try (InputStream publicCertStream = new FileInputStream(certFile.getFileName())) { final CertificateFactory cf = CertificateFactory.getInstance("X.509"); - certificate = (X509Certificate) cf.generateCertificate(publicCertStream); + final List certificateList = cf.generateCertificates(publicCertStream) + .stream().map(o -> (X509Certificate) o).collect(Collectors.toList()); keyStore = KeyStore.getInstance("JKS"); - String alias = certificate.getSubjectX500Principal().getName(); - privateKey = SecurityUtility.loadPrivateKeyFromPemFile(keyFile.getFileName()); + final String alias = certificateList.get(0).getSubjectX500Principal().getName(); + final PrivateKey privateKey = SecurityUtility.loadPrivateKeyFromPemFile(keyFile.getFileName()); keyStore.load(null); - keyStore.setKeyEntry(alias, privateKey, KEYSTORE_PASSWORD, new X509Certificate[] { certificate }); + keyStore.setKeyEntry(alias, privateKey, KEYSTORE_PASSWORD, certificateList.toArray(new Certificate[0])); } catch (IOException | KeyManagementException e) { throw new IllegalArgumentException(e); } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/KeyManagerProxyTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/KeyManagerProxyTest.java new file mode 100644 index 0000000000000..5542f0b22ac95 --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/KeyManagerProxyTest.java @@ -0,0 +1,51 @@ +/* + * 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.common.util; + +import static org.testng.Assert.assertEquals; +import com.google.common.io.Resources; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import lombok.Cleanup; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +public class KeyManagerProxyTest { + + @DataProvider(name = "certDataProvider") + public static Object[][] caDataProvider() { + return new Object[][]{ + {"ca/multiple-ca.pem", 2}, + {"ca/single-ca.pem", 1} + }; + } + + @Test(dataProvider = "certDataProvider") + public void testLoadCert(String path, int certCount) { + final String certFilePath = Resources.getResource(path).getPath(); + // This key is not paired with certs, but this is not a problem as the key is not used in this test + final String keyFilePath = Resources.getResource("ssl/my-ca/client-key.pem").getPath(); + @Cleanup("shutdownNow") + final ScheduledExecutorService scheduledExecutor = Executors.newSingleThreadScheduledExecutor(); + + final KeyManagerProxy keyManager = new KeyManagerProxy(certFilePath, keyFilePath, 60, scheduledExecutor); + assertEquals(keyManager.getCertificateChain("cn=test1").length, certCount); + } +} From bd3b3b82abc78db6641b8917e8a2d2c947459ea7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 28 Nov 2024 04:38:15 +0800 Subject: [PATCH 160/327] [fix][broker] fix null lookup result when brokers are starting (#23642) --- .../apache/pulsar/broker/PulsarService.java | 6 +++--- .../pulsar/broker/lookup/TopicLookupBase.java | 19 +++++++++++++++++++ .../broker/namespace/OwnershipCache.java | 8 +++++++- 3 files changed, 29 insertions(+), 4 deletions(-) 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 05491d9c281c6..a8331d86fc441 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 @@ -979,6 +979,9 @@ public void start() throws PulsarServerException { this.webSocketService.setLocalCluster(clusterData); } + // Initialize namespace service, after service url assigned. Should init zk and refresh self owner info. + this.nsService.initialize(); + // Start the leader election service startLeaderElectionService(); @@ -990,9 +993,6 @@ public void start() throws PulsarServerException { // (namespace service depends on load manager) this.startLoadManagementService(); - // Initialize namespace service, after service url assigned. Should init zk and refresh self owner info. - this.nsService.initialize(); - // Start topic level policies service this.topicPoliciesService = initTopicPoliciesService(); this.topicPoliciesService.start(this); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java index 42f145d32aab1..e01ec83c86018 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java @@ -318,6 +318,7 @@ public static CompletableFuture lookupTopicAsync(PulsarService pulsarSe } LookupData lookupData = lookupResult.get().getLookupData(); + printWarnLogIfLookupResUnexpected(topicName, lookupData, options, pulsarService); if (lookupResult.get().isRedirect()) { boolean newAuthoritative = lookupResult.get().isAuthoritativeRedirect(); lookupfuture.complete( @@ -342,6 +343,24 @@ public static CompletableFuture lookupTopicAsync(PulsarService pulsarSe return lookupfuture; } + /** + * Check if a internal client will get a null lookup result. + */ + private static void printWarnLogIfLookupResUnexpected(TopicName topic, LookupData lookupData, LookupOptions options, + PulsarService pulsar) { + if (!pulsar.getBrokerService().isSystemTopic(topic)) { + return; + } + boolean tlsEnabled = pulsar.getConfig().isBrokerClientTlsEnabled(); + if (!tlsEnabled && StringUtils.isBlank(lookupData.getBrokerUrl())) { + log.warn("[{}] Unexpected lookup result: brokerUrl is required when TLS isn't enabled. options: {}," + + " result {}", topic, options, lookupData); + } else if (tlsEnabled && StringUtils.isBlank(lookupData.getBrokerUrlTls())) { + log.warn("[{}] Unexpected lookup result: brokerUrlTls is required when TLS is enabled. options: {}," + + " result {}", topic, options, lookupData); + } + } + private static void handleLookupError(CompletableFuture lookupFuture, String topicName, String clientAppId, long requestId, Throwable ex){ Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java index 868ed2d9fc2c1..ce68c036a622c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java @@ -72,7 +72,7 @@ public class OwnershipCache { /** * The NamespaceEphemeralData objects that can be associated with the current owner, when the broker is disabled. */ - private final NamespaceEphemeralData selfOwnerInfoDisabled; + private NamespaceEphemeralData selfOwnerInfoDisabled; private final LockManager lockManager; @@ -119,6 +119,9 @@ public OwnershipCache(PulsarService pulsar, NamespaceService namespaceService) { this.pulsar = pulsar; this.ownerBrokerUrl = pulsar.getBrokerServiceUrl(); this.ownerBrokerUrlTls = pulsar.getBrokerServiceUrlTls(); + // At this moment, the variables "webServiceAddress" and "webServiceAddressTls" and so on have not been + // initialized, so we will get an empty "selfOwnerInfo" and an empty "selfOwnerInfoDisabled" here. + // But do not worry, these two fields will be set by the method "refreshSelfOwnerInfo" soon. this.selfOwnerInfo = new NamespaceEphemeralData(ownerBrokerUrl, ownerBrokerUrlTls, pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), false, pulsar.getAdvertisedListeners()); @@ -351,6 +354,9 @@ public synchronized boolean refreshSelfOwnerInfo() { this.selfOwnerInfo = new NamespaceEphemeralData(pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls(), pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), false, pulsar.getAdvertisedListeners()); + this.selfOwnerInfoDisabled = new NamespaceEphemeralData(pulsar.getBrokerServiceUrl(), + pulsar.getBrokerServiceUrlTls(), pulsar.getWebServiceAddress(), + pulsar.getWebServiceAddressTls(), true, pulsar.getAdvertisedListeners()); return selfOwnerInfo.getNativeUrl() != null || selfOwnerInfo.getNativeUrlTls() != null; } } From 9b79d701a2c1067693508515915b5a4504f60916 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 28 Nov 2024 05:55:10 +0800 Subject: [PATCH 161/327] [improve] Use single buffer for metrics when noUnsafe use (#23612) Co-authored-by: Lari Hotari --- build/run_unit_group.sh | 2 + .../PrometheusMetricsGenerator.java | 24 +++++---- ...theusMetricsGeneratorWithNoUnsafeTest.java | 52 +++++++++++++++++++ 3 files changed, 69 insertions(+), 9 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorWithNoUnsafeTest.java diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index cdaf69e351b6d..40f83efaf2fb9 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -77,6 +77,8 @@ alias echo='{ [[ $- =~ .*x.* ]] && trace_enabled=1 || trace_enabled=0; set +x; } # Test Groups -- start -- function test_group_broker_group_1() { mvn_test -pl pulsar-broker -Dgroups='broker' -DtestReuseFork=true + # run tests in broker-isolated group individually (instead of with -Dgroups=broker-isolated) to avoid scanning all test classes + mvn_test -pl pulsar-broker -Dtest=org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGeneratorWithNoUnsafeTest -DtestForkCount=1 -DtestReuseFork=false } function test_group_broker_group_2() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index 8c3cb39c925d7..97d5a7bc9538d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -25,6 +25,7 @@ import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; +import io.netty.util.internal.PlatformDependent; import io.prometheus.client.Collector; import java.io.BufferedOutputStream; import java.io.IOException; @@ -364,19 +365,24 @@ protected ByteBuf generateMetrics(List metricsProv } } - private ByteBuf allocateMultipartCompositeDirectBuffer() { + ByteBuf allocateMultipartCompositeDirectBuffer() { // use composite buffer with pre-allocated buffers to ensure that the pooled allocator can be used // for allocating the buffers ByteBufAllocator byteBufAllocator = PulsarByteBufAllocator.DEFAULT; - int chunkSize = resolveChunkSize(byteBufAllocator); - CompositeByteBuf buf = byteBufAllocator.compositeDirectBuffer( + ByteBuf buf; + if (PlatformDependent.hasUnsafe()) { + int chunkSize = resolveChunkSize(byteBufAllocator); + buf = byteBufAllocator.compositeDirectBuffer( Math.max(MINIMUM_FOR_MAX_COMPONENTS, (initialBufferSize / chunkSize) + 1)); - int totalLen = 0; - while (totalLen < initialBufferSize) { - totalLen += chunkSize; - // increase the capacity in increments of chunkSize to preallocate the buffers - // in the composite buffer - buf.capacity(totalLen); + int totalLen = 0; + while (totalLen < initialBufferSize) { + totalLen += chunkSize; + // increase the capacity in increments of chunkSize to preallocate the buffers + // in the composite buffer + buf.capacity(totalLen); + } + } else { + buf = byteBufAllocator.directBuffer(initialBufferSize); } return buf; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorWithNoUnsafeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorWithNoUnsafeTest.java new file mode 100644 index 0000000000000..006428b4815f1 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorWithNoUnsafeTest.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.stats.prometheus; + +import static org.testng.Assert.assertFalse; +import io.netty.buffer.ByteBuf; +import io.netty.util.internal.PlatformDependent; +import java.time.Clock; +import lombok.Cleanup; +import org.apache.pulsar.common.util.SimpleTextOutputStream; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Test(groups = "broker-isolated") +public class PrometheusMetricsGeneratorWithNoUnsafeTest { + + @BeforeClass + static void setup() { + System.setProperty("io.netty.noUnsafe", "true"); + } + + @Test + public void testWriteStringWithNoUnsafe() { + assertFalse(PlatformDependent.hasUnsafe()); + @Cleanup + PrometheusMetricsGenerator generator = new PrometheusMetricsGenerator(null, false, false, false, false, + Clock.systemUTC()); + @Cleanup("release") + ByteBuf buf = generator.allocateMultipartCompositeDirectBuffer(); + for (int i = 0; i < 2; i++) { + buf.writeBytes(new byte[1024 * 1024]); + } + SimpleTextOutputStream outputStream = new SimpleTextOutputStream(buf); + outputStream.write("test"); + } +} From b284cd4f66a95f01bfd2b501f49809018e2073cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=90=A7=E6=98=93=E5=AE=A2?= Date: Thu, 28 Nov 2024 06:46:05 +0800 Subject: [PATCH 162/327] [fix][client] Make protobuf-java dependency optional in java client libraries (#23632) --- distribution/shell/src/assemble/LICENSE.bin.txt | 4 ---- pulsar-client/pom.xml | 2 +- pulsar-common/pom.xml | 5 ----- 3 files changed, 1 insertion(+), 10 deletions(-) diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 156690f03f249..1601f32bb2b34 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -429,10 +429,6 @@ MIT License * The Checker Framework - checker-qual-3.33.0.jar -Protocol Buffers License - * Protocol Buffers - - protobuf-java-3.25.5.jar -- ../licenses/LICENSE-protobuf.txt - CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - javax.annotation-api-1.3.2.jar diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 37e488d8876eb..49bb3c6490ae9 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -29,7 +29,7 @@ pulsar-client-original - Pulsar Client Java + Pulsar Client Java Original diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 5600aff0ea850..241dd173ea9a4 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -200,11 +200,6 @@ true - - com.google.protobuf - protobuf-java - - com.google.re2j re2j From c50fa566bce873f18502d431ec2f35ac160bf2ca Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Thu, 28 Nov 2024 19:33:21 +0800 Subject: [PATCH 163/327] [improve][pip] PIP-389: Add Producer config compressMinMsgBodySize to improve compression performance (#23526) Co-authored-by: xiangying --- pip/pip-389.md | 110 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 110 insertions(+) create mode 100644 pip/pip-389.md diff --git a/pip/pip-389.md b/pip/pip-389.md new file mode 100644 index 0000000000000..9b23ce4341e8a --- /dev/null +++ b/pip/pip-389.md @@ -0,0 +1,110 @@ +# PIP-389: Add Producer config compressMinMsgBodySize to improve compression performance + +# Background knowledge +Pulsar provide a way to compress messages before sending them to the broker[0]. This can be done by setting the `compressionType` in the producer configuration. +The compressionType can be set to one of the following values: +- LZ4 +- ZLIB +- ZSTD +- SNAPPY + +But the compressionType is applied to all messages sent by the producer. This means that even small messages are compressed. + +In our test, we found that compressing small messages can is meaningless. The compression ratio is low and spend more cpu. +The relevant description in the official documentation: +>The smaller the amount of data to compress, the more difficult it is to compress. This problem is common to all compression algorithms. [1] + +The similar configuration in RocketMQ is `compressMsgBodyOverHowmuch`[2]: +>/** +>* Compress message body threshold, namely, message body larger than 4k will be compressed on default. +>*/ +>private int compressMsgBodyOverHowmuch = 1024 * 4; + +[0] https://pulsar.apache.org/docs/4.0.x/concepts-messaging/#compression +[1] https://github.com/facebook/zstd?tab=readme-ov-file#the-case-for-small-data-compression +[2] https://github.com/apache/rocketmq/blob/dd62ed0f3b16919adec5d5eece21a1050dc9c5a0/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java#L117 + +# Motivation + +The motivation of this PIP is to provide a way to improve the compression performance by skipping the compression of small messages. +We want to add a new configuration `compressMinMsgBodySize` to the producer configuration. +This configuration will allow the user to set the minimum size of the message body that will be compressed. +If the message body size is less than the `compressMinMsgBodySize`, the message will not be compressed. + +# Goals + +## In Scope + +Add a new configuration `compressMinMsgBodySize` to the producer configuration. + +## Out of Scope + +Solve the compression problem of small data + +# High Level Design + +# Detailed Design + +## Design & Implementation Details + +Add a new configuration `compressMinMsgBodySize` to the producer configuration. +This configuration will allow the user to set the minimum size of the message body that will be compressed. +If the message body size is less than the `compressMinMsgBodySize`, the message will not be compressed. + +## Public-facing Changes + +Add a new configuration `compressMinMsgBodySize` to the producer configuration. + + +### Public API +NA +### Binary protocol + +### Configuration + +### CLI + +### Metrics + +NA + +# Monitoring + +NA + +# Security Considerations + +NA + +# Backward & Forward Compatibility + +## Upgrade + +This is a new feature, and it does not affect the existing configuration. + +## Downgrade / Rollback + +The new configuration `compressMinMsgBodySize` will to be removed from the producer configuration. +If you used it, you need to remove it manually. + +## Pulsar Geo-Replication Upgrade & Downgrade/Rollback Considerations + + + +# Alternatives + + + +# General Notes + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/vxvy7h61hg9wlgby6lcpkm9osdk9sx20 +* Mailing List voting thread: https://lists.apache.org/thread/xv7x3vmycxzsrhbdo7vmssh8lxxzyxd5 From 963be2c3290a4a6ef25b549b046039afc8690d4c Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 28 Nov 2024 21:44:15 +0800 Subject: [PATCH 164/327] [improve][broker] Decouple pulsar_storage_backlog_age_seconds metric with backlogQuota check (#23619) --- .../pulsar/broker/service/BrokerService.java | 44 +++-- .../service/persistent/PersistentTopic.java | 184 +++++++++--------- .../service/persistent/SystemTopic.java | 2 +- .../service/BacklogQuotaManagerTest.java | 133 ++++++++++++- 4 files changed, 241 insertions(+), 122 deletions(-) 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 77cd52f4558ea..6afa1ae32fbcb 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 @@ -2245,29 +2245,31 @@ public BacklogQuotaManager getBacklogQuotaManager() { public void monitorBacklogQuota() { long startTimeMillis = System.currentTimeMillis(); forEachPersistentTopic(topic -> { - if (topic.isSizeBacklogExceeded()) { - getBacklogQuotaManager().handleExceededBacklogQuota(topic, - BacklogQuota.BacklogQuotaType.destination_storage, false); - } else { - topic.checkTimeBacklogExceeded().thenAccept(isExceeded -> { - if (isExceeded) { - getBacklogQuotaManager().handleExceededBacklogQuota(topic, - BacklogQuota.BacklogQuotaType.message_age, - pulsar.getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()); - } else { - if (log.isDebugEnabled()) { - log.debug("quota not exceeded for [{}]", topic.getName()); + topic.updateOldPositionInfo().thenAccept(__ -> { + if (topic.isSizeBacklogExceeded()) { + getBacklogQuotaManager().handleExceededBacklogQuota(topic, + BacklogQuota.BacklogQuotaType.destination_storage, false); + } else { + topic.checkTimeBacklogExceeded(false).thenAccept(isExceeded -> { + if (isExceeded) { + getBacklogQuotaManager().handleExceededBacklogQuota(topic, + BacklogQuota.BacklogQuotaType.message_age, + pulsar.getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()); + } else { + if (log.isDebugEnabled()) { + log.debug("quota not exceeded for [{}]", topic.getName()); + } } - } - }).exceptionally(throwable -> { - log.error("Error when checkTimeBacklogExceeded({}) in monitorBacklogQuota", + }); + } + }).whenComplete((unused, throwable) -> { + if (throwable != null) { + log.error("Error when checkBacklogQuota({}) in monitorBacklogQuota", topic.getName(), throwable); - return null; - }).whenComplete((unused, throwable) -> { - backlogQuotaCheckDuration.observe( - MILLISECONDS.toSeconds(System.currentTimeMillis() - startTimeMillis)); - }); - } + } + backlogQuotaCheckDuration.observe( + MILLISECONDS.toSeconds(System.currentTimeMillis() - startTimeMillis)); + }); }); } 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 651d12373628b..eb48ceee72d76 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 @@ -295,14 +295,15 @@ protected TopicStatsHelper initialValue() { PERSISTENT_TOPIC_ATTRIBUTES_FIELD_UPDATER = AtomicReferenceFieldUpdater.newUpdater( PersistentTopic.class, PersistentTopicAttributes.class, "persistentTopicAttributes"); - private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; - private static final AtomicReferenceFieldUpdater + // The topic's oldest position information, if null, indicates that there is no cursor or no backlog. + private volatile OldestPositionInfo oldestPositionInfo; + private static final AtomicReferenceFieldUpdater TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( PersistentTopic.class, - TimeBasedBacklogQuotaCheckResult.class, - "timeBasedBacklogQuotaCheckResult"); + OldestPositionInfo.class, + "oldestPositionInfo"); @Value - private static class TimeBasedBacklogQuotaCheckResult { + private static class OldestPositionInfo { Position oldestCursorMarkDeletePosition; String cursorName; long positionPublishTimestampInMillis; @@ -2634,12 +2635,10 @@ public CompletableFuture asyncGetStats(GetStatsOptions stats.backlogQuotaLimitSize = getBacklogQuota(BacklogQuotaType.destination_storage).getLimitSize(); stats.backlogQuotaLimitTime = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); - TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; stats.oldestBacklogMessageAgeSeconds = getBestEffortOldestUnacknowledgedMessageAgeSeconds(); - stats.oldestBacklogMessageSubscriptionName = (backlogQuotaCheckResult == null) - || !hasBacklogs(getStatsOptions.isGetPreciseBacklog()) + stats.oldestBacklogMessageSubscriptionName = (oldestPositionInfo == null) ? null - : backlogQuotaCheckResult.getCursorName(); + : oldestPositionInfo.getCursorName(); stats.compaction.reset(); mxBean.flatMap(bean -> bean.getCompactionRecordForTopic(topic)).map(compactionRecord -> { @@ -3425,7 +3424,7 @@ public CompletableFuture checkBacklogQuotaExceeded(String producerName, Ba return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } if (backlogQuotaType == BacklogQuotaType.message_age) { - return checkTimeBacklogExceeded().thenCompose(isExceeded -> { + return checkTimeBacklogExceeded(true).thenCompose(isExceeded -> { if (isExceeded) { log.debug("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); @@ -3463,19 +3462,15 @@ public boolean isSizeBacklogExceeded() { @Override public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { - if (!hasBacklogs(false)) { - return 0; - } - TimeBasedBacklogQuotaCheckResult result = timeBasedBacklogQuotaCheckResult; - if (result == null) { + if (oldestPositionInfo == null) { return -1; } else { return TimeUnit.MILLISECONDS.toSeconds( - Clock.systemUTC().millis() - result.getPositionPublishTimestampInMillis()); + Clock.systemUTC().millis() - oldestPositionInfo.getPositionPublishTimestampInMillis()); } } - private void updateResultIfNewer(TimeBasedBacklogQuotaCheckResult updatedResult) { + private void updateResultIfNewer(OldestPositionInfo updatedResult) { TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.updateAndGet(this, existingResult -> { if (existingResult == null @@ -3489,74 +3484,56 @@ private void updateResultIfNewer(TimeBasedBacklogQuotaCheckResult updatedResult) } - /** - * @return determine if backlog quota enforcement needs to be done for topic based on time limit - */ - public CompletableFuture checkTimeBacklogExceeded() { + public CompletableFuture updateOldPositionInfo() { TopicName topicName = TopicName.get(getName()); - int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); - if (log.isDebugEnabled()) { - log.debug("[{}] Time backlog quota = [{}]. Checking if exceeded.", topicName, backlogQuotaLimitInSecond); - } - // If backlog quota by time is not set - if (backlogQuotaLimitInSecond <= 0) { - return CompletableFuture.completedFuture(false); + if (!(ledger.getCursors() instanceof ManagedCursorContainer managedCursorContainer)) { + return CompletableFuture.failedFuture(new IllegalStateException( + String.format("[%s] No valid cursors found. Skip update old position info.", topicName))); } - ManagedCursorContainer managedCursorContainer = (ManagedCursorContainer) ledger.getCursors(); - CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestPosition(); + if (!hasBacklogs(brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck())) { + if (log.isDebugEnabled()) { + log.debug("[{}] No backlog. Update old position info is null", topicName); + } + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.set(this, null); + return CompletableFuture.completedFuture(null); + } // If we have no durable cursor since `ledger.getCursors()` only managed durable cursors - if (oldestMarkDeleteCursorInfo == null - || oldestMarkDeleteCursorInfo.getPosition() == null) { + CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestPosition(); + if (oldestMarkDeleteCursorInfo == null || oldestMarkDeleteCursorInfo.getPosition() == null) { if (log.isDebugEnabled()) { - log.debug("[{}] No durable cursor found. Skipping time based backlog quota check." - + " Oldest mark-delete cursor info: {}", topicName, oldestMarkDeleteCursorInfo); + log.debug("[{}] No durable cursor found. Update old position info is null", topicName); } - return CompletableFuture.completedFuture(false); + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.set(this, null); + return CompletableFuture.completedFuture(null); } Position oldestMarkDeletePosition = oldestMarkDeleteCursorInfo.getPosition(); - - TimeBasedBacklogQuotaCheckResult lastCheckResult = timeBasedBacklogQuotaCheckResult; - if (lastCheckResult != null - && oldestMarkDeletePosition.compareTo(lastCheckResult.getOldestCursorMarkDeletePosition()) == 0) { - + OldestPositionInfo lastOldestPositionInfo = oldestPositionInfo; + if (lastOldestPositionInfo != null + && oldestMarkDeletePosition.compareTo(lastOldestPositionInfo.getOldestCursorMarkDeletePosition()) == 0) { // Same position, but the cursor causing it has changed? - if (!lastCheckResult.getCursorName().equals(oldestMarkDeleteCursorInfo.getCursor().getName())) { - final TimeBasedBacklogQuotaCheckResult updatedResult = new TimeBasedBacklogQuotaCheckResult( - lastCheckResult.getOldestCursorMarkDeletePosition(), + if (!lastOldestPositionInfo.getCursorName().equals(oldestMarkDeleteCursorInfo.getCursor().getName())) { + updateResultIfNewer(new OldestPositionInfo( + lastOldestPositionInfo.getOldestCursorMarkDeletePosition(), oldestMarkDeleteCursorInfo.getCursor().getName(), - lastCheckResult.getPositionPublishTimestampInMillis(), - oldestMarkDeleteCursorInfo.getVersion()); - - updateResultIfNewer(updatedResult); + lastOldestPositionInfo.getPositionPublishTimestampInMillis(), + oldestMarkDeleteCursorInfo.getVersion())); if (log.isDebugEnabled()) { - log.debug("[{}] Time-based backlog quota check. Updating cached result for position {}, " - + "since cursor causing it has changed from {} to {}", + log.debug("[{}] Updating cached old position info {}, " + + "since cursor causing it has changed from {} to {}", topicName, oldestMarkDeletePosition, - lastCheckResult.getCursorName(), + lastOldestPositionInfo.getCursorName(), oldestMarkDeleteCursorInfo.getCursor().getName()); } } - - long entryTimestamp = lastCheckResult.getPositionPublishTimestampInMillis(); - boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); - if (log.isDebugEnabled()) { - log.debug("[{}] Time based backlog quota check. Using cache result for position {}. " - + "Entry timestamp: {}, expired: {}", - topicName, oldestMarkDeletePosition, entryTimestamp, expired); - } - return CompletableFuture.completedFuture(expired); + return CompletableFuture.completedFuture(null); } - if (brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()) { - if (!hasBacklogs(true)) { - return CompletableFuture.completedFuture(false); - } - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. Position position = ledger.getNextValidPosition(oldestMarkDeletePosition); @@ -3566,34 +3543,28 @@ public CompletableFuture checkTimeBacklogExceeded() { public void readEntryComplete(Entry entry, Object ctx) { try { long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); - updateResultIfNewer( - new TimeBasedBacklogQuotaCheckResult( - oldestMarkDeleteCursorInfo.getPosition(), - oldestMarkDeleteCursorInfo.getCursor().getName(), - entryTimestamp, - oldestMarkDeleteCursorInfo.getVersion())); - - boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); + new OldestPositionInfo( + oldestMarkDeleteCursorInfo.getPosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + entryTimestamp, + oldestMarkDeleteCursorInfo.getVersion())); if (log.isDebugEnabled()) { - log.debug("[{}] Time based backlog quota check. Oldest unacked entry read from BK. " + log.debug("[{}] Precise based update oldest position info. " + + "Oldest unacked entry read from BK. " + "Oldest entry in cursor {}'s backlog: {}. " + "Oldest mark-delete position: {}. " - + "Quota {}. Last check result position [{}]. " - + "Expired: {}, entryTimestamp: {}", + + "EntryTimestamp: {}", topicName, oldestMarkDeleteCursorInfo.getCursor().getName(), position, oldestMarkDeletePosition, - backlogQuotaLimitInSecond, - lastCheckResult.getOldestCursorMarkDeletePosition(), - expired, entryTimestamp); } - future.complete(expired); + future.complete(null); } catch (Exception e) { - log.error("[{}][{}] Error deserializing message for backlog check", topicName, e); - future.complete(false); + log.error("[{}][{}] Error deserializing message for update old position", topicName, e); + future.completeExceptionally(e); } finally { entry.release(); } @@ -3601,36 +3572,61 @@ public void readEntryComplete(Entry entry, Object ctx) { @Override public void readEntryFailed(ManagedLedgerException exception, Object ctx) { - log.error("[{}][{}] Error reading entry for precise time based backlog check", + log.error("[{}][{}] Error reading entry for precise update old position", topicName, exception); - future.complete(false); + future.completeExceptionally(exception); } }, null); return future; } else { try { - if (!hasBacklogs(false)) { - return CompletableFuture.completedFuture(false); - } EstimateTimeBasedBacklogQuotaCheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { updateResultIfNewer( - new TimeBasedBacklogQuotaCheckResult( - oldestMarkDeleteCursorInfo.getPosition(), - oldestMarkDeleteCursorInfo.getCursor().getName(), - checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp(), - oldestMarkDeleteCursorInfo.getVersion())); + new OldestPositionInfo( + oldestMarkDeleteCursorInfo.getPosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp(), + oldestMarkDeleteCursorInfo.getVersion())); } - return CompletableFuture.completedFuture(checkResult.isTruncateBacklogToMatchQuota()); + return CompletableFuture.completedFuture(null); } catch (Exception e) { - log.error("[{}][{}] Error reading entry for precise time based backlog check", topicName, e); - return CompletableFuture.completedFuture(false); + log.error("[{}][{}] Error reading entry for update old position", topicName, e); + return CompletableFuture.failedFuture(e); } } } + /** + * @return determine if backlog quota enforcement needs to be done for topic based on time limit + */ + public CompletableFuture checkTimeBacklogExceeded(boolean shouldUpdateOldPositionInfo) { + TopicName topicName = TopicName.get(getName()); + int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + + if (log.isDebugEnabled()) { + log.debug("[{}] Time backlog quota = [{}]. Checking if exceeded.", topicName, backlogQuotaLimitInSecond); + } + CompletableFuture updateFuture = shouldUpdateOldPositionInfo ? updateOldPositionInfo() + : CompletableFuture.completedFuture(null); + return updateFuture.thenCompose(__ -> { + if (backlogQuotaLimitInSecond <= 0) { + return CompletableFuture.completedFuture(false); + } + if (oldestPositionInfo == null) { + return CompletableFuture.completedFuture(false); + } + long entryTimestamp = oldestPositionInfo.getPositionPublishTimestampInMillis(); + boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); + return CompletableFuture.completedFuture(expired); + }).exceptionally(e -> { + log.error("[{}][{}] Error checking time backlog exceeded", topicName, e); + return false; + }); + } + private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( Position markDeletePosition) throws ExecutionException, InterruptedException { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java index 8feb432a08001..a26255c9f8bad 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java @@ -51,7 +51,7 @@ public boolean isSizeBacklogExceeded() { } @Override - public CompletableFuture checkTimeBacklogExceeded() { + public CompletableFuture checkTimeBacklogExceeded(boolean shouldUpdateOldPositionInfo) { return CompletableFuture.completedFuture(false); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index 56f9f4f91246e..963dc3d26b28b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -526,7 +526,7 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce assertThat(topicStats.getBacklogSize()).isEqualTo(0); assertThat(topicStats.getSubscriptions().get(subName1).getMsgBacklog()).isEqualTo(0); assertThat(topicStats.getSubscriptions().get(subName2).getMsgBacklog()).isEqualTo(0); - assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(-1); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); metrics = prometheusMetricsClient.getMetrics(); @@ -537,7 +537,7 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce entry("cluster", CLUSTER_NAME), entry("namespace", namespace), entry("topic", topic1)); - assertThat((long) backlogAgeMetric.value).isEqualTo(0); + assertThat((long) backlogAgeMetric.value).isEqualTo(-1); // producer should create success. Producer producer2 = createProducer(client, topic1); @@ -598,7 +598,7 @@ public void backlogsStatsPreciseWithNoBacklog() throws PulsarAdminException, Pul assertThat(topicStats.getBacklogQuotaLimitTime()).isEqualTo(timeLimitSeconds); assertThat(topicStats.getBacklogSize()).isEqualTo(0); assertThat(topicStats.getSubscriptions().get(subName1).getMsgBacklog()).isEqualTo(0); - assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(-1); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); Metric backlogAgeMetric = @@ -608,7 +608,7 @@ public void backlogsStatsPreciseWithNoBacklog() throws PulsarAdminException, Pul entry("cluster", CLUSTER_NAME), entry("namespace", namespace), entry("topic", topic1)); - assertThat((long) backlogAgeMetric.value).isEqualTo(0); + assertThat((long) backlogAgeMetric.value).isEqualTo(-1); // producer should create success. Producer producer2 = createProducer(client, topic1); @@ -618,6 +618,118 @@ public void backlogsStatsPreciseWithNoBacklog() throws PulsarAdminException, Pul config.setExposePreciseBacklogInPrometheus(false); } + @Test + public void backlogsAgeMetricsPreciseWithoutBacklogQuota() throws Exception { + config.setPreciseTimeBasedBacklogQuotaCheck(true); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .statsInterval(0, SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "c1"; + final String subName2 = "c2"; + final int numMsgs = 4; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .subscribe(); + Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2) + .acknowledgmentGroupTime(0, SECONDS) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + Thread.sleep(3000); // Guarantees if we use wrong message in age, to show up in failed test + producer.send(content); + } + + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + + // Move subscription 1, one message, such that subscription 2 is the oldest + // S2 S1 + // 0 1 + Message oldestMessage = consumer1.receive(); + consumer1.acknowledge(oldestMessage); + log.info("Subscription 1 moved 1 message. Now subscription 2 is the oldest. Oldest message:"+ + oldestMessage.getMessageId()); + + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + TopicStats topicStats = getTopicStats(topic1); + + long expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()) + .isCloseTo(expectedMessageAgeSeconds, within(1L)); + + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.tags).containsExactly( + entry("cluster", CLUSTER_NAME), + entry("namespace", namespace), + entry("topic", topic1)); + assertThat((long) backlogAgeMetric.value).isCloseTo(expectedMessageAgeSeconds, within(2L)); + } + config.setPreciseTimeBasedBacklogQuotaCheck(false); + } + + @Test + public void backlogsAgeMetricsNoPreciseWithoutBacklogQuota() throws Exception { + config.setPreciseTimeBasedBacklogQuotaCheck(false); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .statsInterval(0, SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "c1"; + final int numMsgs = 5; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + Thread.sleep(3000); // Guarantees if we use wrong message in age, to show up in failed test + producer.send(content); + } + + Message oldestMessage = consumer1.receive(); + consumer1.acknowledge(oldestMessage); + log.info("Moved subscription 1, by 1 message"); + + // Unload topic to trigger the ledger close + unloadAndLoadTopic(topic1, producer); + long unloadTime = System.currentTimeMillis(); + waitForQuotaCheckToRunTwice(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + TopicStats topicStats = getTopicStats(topic1); + + long expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()) + .isCloseTo(expectedMessageAgeSeconds, within(1L)); + + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.tags).containsExactly( + entry("cluster", CLUSTER_NAME), + entry("namespace", namespace), + entry("topic", topic1)); + assertThat((long) backlogAgeMetric.value).isCloseTo(expectedMessageAgeSeconds, within(2L)); + } + } + private long getReadEntries(String topic1) { return ((PersistentTopic) pulsar.getBrokerService().getTopicReference(topic1).get()) .getManagedLedger().getStats().getEntriesReadTotalCount(); @@ -718,6 +830,15 @@ public void backlogsStatsNotPrecise() throws PulsarAdminException, PulsarClientE assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); expectedAge = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedAge, within(1L)); + + // Unsubscribe consume1 and consumer2 + consumer1.unsubscribe(); + consumer2.unsubscribe(); + waitForQuotaCheckToRunTwice(); + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(-1); + config.setManagedLedgerMaxEntriesPerLedger(MAX_ENTRIES_PER_LEDGER); } } @@ -773,11 +894,11 @@ public void backlogsStatsNotPreciseWithNoBacklog() throws PulsarAdminException, Metrics metrics = prometheusMetricsClient.getMetrics(); assertEquals(topicStats.getSubscriptions().get(subName1).getMsgBacklog(), 0); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); - assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(-1); Metric backlogAgeMetric = metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", Pair.of("topic", topic1)); - assertThat(backlogAgeMetric.value).isEqualTo(0); + assertThat(backlogAgeMetric.value).isEqualTo(-1); // producer should create success. Producer producer2 = createProducer(client, topic1); From 68eb8f29c50f16f0aa50ecacec16882b82230122 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Thu, 28 Nov 2024 22:58:24 +0800 Subject: [PATCH 165/327] [fix][client] Fix deadlock of NegativeAcksTracker (#23651) --- .../client/impl/NegativeAcksTracker.java | 40 ++++++++++--------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java index e1724ebb85cda..5256ebf04f43c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java @@ -68,36 +68,38 @@ public NegativeAcksTracker(ConsumerBase consumer, ConsumerConfigurationData messagesToRedeliver = new HashSet<>(); - long now = System.nanoTime(); - nackedMessages.forEach((ledgerId, entryId, partitionIndex, timestamp) -> { - if (timestamp < now) { - MessageId msgId = new MessageIdImpl(ledgerId, entryId, - // need to covert non-partitioned topic partition index to -1 - (int) (partitionIndex == NON_PARTITIONED_TOPIC_PARTITION_INDEX ? -1 : partitionIndex)); - addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); - messagesToRedeliver.add(msgId); + synchronized (this) { + if (nackedMessages.isEmpty()) { + this.timeout = null; + return; } - }); - if (!messagesToRedeliver.isEmpty()) { + long now = System.nanoTime(); + nackedMessages.forEach((ledgerId, entryId, partitionIndex, timestamp) -> { + if (timestamp < now) { + MessageId msgId = new MessageIdImpl(ledgerId, entryId, + // need to covert non-partitioned topic partition index to -1 + (int) (partitionIndex == NON_PARTITIONED_TOPIC_PARTITION_INDEX ? -1 : partitionIndex)); + addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); + messagesToRedeliver.add(msgId); + } + }); for (MessageId messageId : messagesToRedeliver) { nackedMessages.remove(((MessageIdImpl) messageId).getLedgerId(), ((MessageIdImpl) messageId).getEntryId()); } + this.timeout = timer.newTimeout(this::triggerRedelivery, timerIntervalNanos, TimeUnit.NANOSECONDS); + } + + // release the lock of NegativeAcksTracker before calling consumer.redeliverUnacknowledgedMessages, + // in which we may acquire the lock of consumer, leading to potential deadlock. + if (!messagesToRedeliver.isEmpty()) { consumer.onNegativeAcksSend(messagesToRedeliver); log.info("[{}] {} messages will be re-delivered", consumer, messagesToRedeliver.size()); consumer.redeliverUnacknowledgedMessages(messagesToRedeliver); } - - this.timeout = timer.newTimeout(this::triggerRedelivery, timerIntervalNanos, TimeUnit.NANOSECONDS); } public synchronized void add(MessageId messageId) { From 7e6fa554012b237308db1bebf3c625e103bf0db5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 28 Nov 2024 18:01:17 +0200 Subject: [PATCH 166/327] [improve] Upgrade oxia-java to 0.4.10 and fix closing of OxiaMetadataStore (#23653) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- .../pulsar/metadata/impl/oxia/OxiaMetadataStore.java | 8 +++++--- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 7c66460c21656..fd393cfec9b76 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -481,8 +481,8 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-api-0.4.9.jar - - io.streamnative.oxia-oxia-client-0.4.9.jar + - io.streamnative.oxia-oxia-client-api-0.4.10.jar + - io.streamnative.oxia-oxia-client-0.4.10.jar * OpenHFT - net.openhft-zero-allocation-hashing-0.16.jar * Java JSON WebTokens diff --git a/pom.xml b/pom.xml index 40afef1241a54..de05eae5755c0 100644 --- a/pom.xml +++ b/pom.xml @@ -251,7 +251,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.7 - 0.4.9 + 0.4.10 2.0 1.10.12 5.5.0 diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java index 27cd4a2d2f60b..4073eb8dcca2a 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java @@ -297,10 +297,12 @@ private CompletableFuture createParents(String path) { @Override public void close() throws Exception { - if (client != null) { - client.close(); + if (isClosed.compareAndSet(false, true)) { + if (client != null) { + client.close(); + } + super.close(); } - super.close(); } public Optional getMetadataEventSynchronizer() { From 66a8f739bcecc4b719adc9a063d0b17ec766d76e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 29 Nov 2024 04:57:58 +0200 Subject: [PATCH 167/327] [fix][build] Fix error "Element encoding is not allowed here" in pom.xml (#23655) --- pom.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index de05eae5755c0..e6d154c1b34d6 100644 --- a/pom.xml +++ b/pom.xml @@ -2432,7 +2432,8 @@ flexible messaging model and an intuitive client API. ${pulsar.basedir}/buildtools/src/main/resources/pulsar/checkstyle.xml ${pulsar.basedir}/buildtools/src/main/resources/pulsar/suppressions.xml - UTF-8 + UTF-8 + UTF-8 **/proto/* @@ -2496,7 +2497,7 @@ flexible messaging model and an intuitive client API. ${pulsar.basedir}/buildtools/src/main/resources/pulsar/checkstyle.xml ${pulsar.basedir}/buildtools/src/main/resources/pulsar/suppressions.xml - UTF-8 + UTF-8 **/proto/* From 32b3ccfd331a1a6093aff80b78512dfd0809992f Mon Sep 17 00:00:00 2001 From: zhou zhuohan <843520313@qq.com> Date: Fri, 29 Nov 2024 10:59:19 +0800 Subject: [PATCH 168/327] [improve][client] Replace NameUtil#generateRandomName with RandomStringUtils#randomAlphanumeric (#23645) --- .../pulsar/broker/service/ServerCnxTest.java | 10 +++---- .../pulsar/client/impl/ConsumerBase.java | 5 ++-- .../pulsar/client/impl/ConsumerImpl.java | 4 +-- .../client/impl/MultiTopicsConsumerImpl.java | 6 ++-- .../apache/pulsar/client/util/NameUtil.java | 28 ------------------- .../pulsar/client/impl/ConsumerImplTest.java | 7 +++++ 6 files changed, 20 insertions(+), 40 deletions(-) delete mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 59e9847b75a33..b1c99940827c8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -86,6 +86,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -108,7 +109,6 @@ import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.ProducerAccessMode; import org.apache.pulsar.client.api.transaction.TxnID; -import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.api.proto.AuthMethod; import org.apache.pulsar.common.api.proto.BaseCommand; @@ -1084,8 +1084,8 @@ public void testHandleConsumerAfterClientChannelInactive() throws Exception { final long consumerId = 1; final MutableInt requestId = new MutableInt(1); final String sName = successSubName; - final String cName1 = NameUtil.generateRandomName(); - final String cName2 = NameUtil.generateRandomName(); + final String cName1 = RandomStringUtils.randomAlphanumeric(5); + final String cName2 = RandomStringUtils.randomAlphanumeric(5); resetChannel(); setChannelConnected(); @@ -1126,8 +1126,8 @@ public void test2ndSubFailedIfDisabledConCheck() final long consumerId = 1; final MutableInt requestId = new MutableInt(1); final String sName = successSubName; - final String cName1 = NameUtil.generateRandomName(); - final String cName2 = NameUtil.generateRandomName(); + final String cName1 = RandomStringUtils.randomAlphanumeric(5); + final String cName2 = RandomStringUtils.randomAlphanumeric(5); // Disabled connection check. pulsar.getConfig().setConnectionLivenessCheckTimeoutMillis(-1); resetChannel(); 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 31aef2fd25abb..1ad8c6d28f1d7 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 @@ -43,6 +43,7 @@ import java.util.concurrent.locks.ReentrantLock; import lombok.Getter; import lombok.Setter; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.pulsar.client.api.BatchReceivePolicy; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -61,7 +62,6 @@ import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ExecutorProvider; -import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.client.util.NoOpLock; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandSubscribe; @@ -132,7 +132,8 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat this.maxReceiverQueueSize = receiverQueueSize; this.subscription = conf.getSubscriptionName(); this.conf = conf; - this.consumerName = conf.getConsumerName() == null ? NameUtil.generateRandomName() : conf.getConsumerName(); + this.consumerName = + conf.getConsumerName() == null ? RandomStringUtils.randomAlphanumeric(5) : conf.getConsumerName(); this.subscribeFuture = subscribeFuture; this.listener = conf.getMessageListener(); this.consumerEventListener = conf.getConsumerEventListener(); 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 d46f4af1be748..390a70095182f 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 @@ -69,6 +69,7 @@ import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.Getter; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Triple; import org.apache.pulsar.client.api.Consumer; @@ -98,7 +99,6 @@ import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ExecutorProvider; -import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.client.util.RetryMessageUtil; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.EncryptionContext; @@ -2267,7 +2267,7 @@ private void initDeadLetterProducerIfNeeded() { .initialSubscriptionName(this.deadLetterPolicy.getInitialSubscriptionName()) .topic(this.deadLetterPolicy.getDeadLetterTopic()) .producerName(String.format("%s-%s-%s-%s-DLQ", this.topicName, this.subscription, - this.consumerName, NameUtil.generateRandomName())) + this.consumerName, RandomStringUtils.randomAlphanumeric(5))) .blockIfQueueFull(false) .enableBatching(false) .enableChunking(true) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index dd6a304d9985c..6f9c5b47c55bb 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -53,6 +53,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import javax.annotation.Nullable; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.BatchReceivePolicy; import org.apache.pulsar.client.api.Consumer; @@ -69,7 +70,6 @@ import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ExecutorProvider; -import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.TopicName; @@ -113,7 +113,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { MultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { - this(client, DUMMY_TOPIC_NAME_PREFIX + NameUtil.generateRandomName(), conf, executorProvider, + this(client, DUMMY_TOPIC_NAME_PREFIX + RandomStringUtils.randomAlphanumeric(5), conf, executorProvider, subscribeFuture, schema, interceptors, createTopicIfDoesNotExist); } @@ -121,7 +121,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist, MessageId startMessageId, long startMessageRollbackDurationInSec) { - this(client, DUMMY_TOPIC_NAME_PREFIX + NameUtil.generateRandomName(), conf, executorProvider, + this(client, DUMMY_TOPIC_NAME_PREFIX + RandomStringUtils.randomAlphanumeric(5), conf, executorProvider, subscribeFuture, schema, interceptors, createTopicIfDoesNotExist, startMessageId, startMessageRollbackDurationInSec); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java b/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java deleted file mode 100644 index 4c416b6152cf8..0000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.client.util; - -import java.util.UUID; -import org.apache.commons.codec.digest.DigestUtils; - -public class NameUtil { - public static String generateRandomName() { - return DigestUtils.sha1Hex(UUID.randomUUID().toString()).substring(0, 5); - } -} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java index e62958eb96887..4831f1e384d4b 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java @@ -38,6 +38,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; import lombok.Cleanup; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -296,4 +297,10 @@ public void testSeekAsyncInternal() { assertTrue(secondResult.isCompletedExceptionally()); verify(cnx, times(1)).sendRequestWithId(any(ByteBuf.class), anyLong()); } + + @Test(invocationTimeOut = 1000) + public void testAutoGenerateConsumerName() { + Pattern consumerNamePattern = Pattern.compile("[a-zA-Z0-9]{5}"); + assertTrue(consumerNamePattern.matcher(consumer.getConsumerName()).matches()); + } } From 3a502552f5cbe3717c1039028a8e07d2abef4b06 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 29 Nov 2024 11:03:23 +0800 Subject: [PATCH 169/327] [improve] [pip] PIP-373: Add a topic's system prop that indicates whether users have published TXN messages in before. (#23210) [improve] [pip] PIP-373: Add a topic's system prop that indicates whether users have published TXN messages in before. (#23210) --- pip/pip-373.md | 78 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 78 insertions(+) create mode 100644 pip/pip-373.md diff --git a/pip/pip-373.md b/pip/pip-373.md new file mode 100644 index 0000000000000..b40824a186a73 --- /dev/null +++ b/pip/pip-373.md @@ -0,0 +1,78 @@ +# PIP-373: Add a topic's system prop that indicates whether users have published TXN messages in before. + +# Background knowledge + +**Transaction Buffer** +- It maintains `aborted` messages indexes and `ongoing` TXN messages indexes in memory, we call them `Two Indexes` in the following sections. +- It helps to filter out the messages that have been aborted or are still binding with an `on-going` TXN, to avoid consuming aborted messages or TXN in-progress messages. + +**Transaction Buffer initializes when a topic is loading up** +- It reads messages that contain TXN state from the original topic, to re-build `Two Indexes` into memory. + +**Transaction Buffer closing when a topic is unloading** +- It takes a snapshot for `Two Indexes` to rebuild the states faster next loading up. + +# Motivation + +- Since TXN is a feature that across multiple namespaces, it can be enabled or disabled at the cluster level. +- Transaction Buffer will try to re-build `Two Indexes` for every topic if you enable TXN. +- If you have a huge number of topics in a namespace, the task that re-build `Two Indexes` costs huge resources(CPU, Memory). + +We'd better skip Transaction Buffer re-building `Two Indexes` if the topic it is related to does not contain TXN messages to save resources usage. + +# Goals + +Skip Transaction Buffer re-building `Two Indexes` if the topic it is related to does not contain TXN messages to save resources usage. + +## In Scope + +This PIP only focuses on improving the scenario in which users have never published TXN messages on a topic. + +## Out of Scope + +This PIP does not focus on the scenario that follows. +- Published TXN messages. +- Consumed all TXN messages that were sent before. +- Transaction Buffer keeps re-building `Two Indexes` even if there are no TXN messages in the topic anymore, and it costs resources. + +# Detailed Design + +## Design & Implementation Details + +- Add a topic-level system property named `__contains_txn_messages`, the default value is `false`, and it will be set to `true` when the first TXN message is publishing. +- Transaction Buffer skips re-building `Two Indexes` if the property is `false`. + +## Public-facing Changes + +The topic property `__contains_txn_messages` becomes to a system property, it can never be used by users anymore. + +### Public API +- You will get a `NotAllowedException` when you try to set/remove a topic property named `__contains_txn_messages` by the API `pulsar-admin topics update-properties/remove-properties`. +- The value of the property `__contains_txn_messages` can be queried by `pulsar-admin topics get-properties`. + + +### Metrics +| Name | Description | Attributes | Units| +|--------------------------------------------|-----------------------------------------------------| --- | --- | +| `pulsar_broker_using_txn_topics_count` | Counter. The number of topics contains TXN messages. | cluster | - | + +# Backward & Forward Compatibility + +## Upgrade + +There are `3` scenarios that the topic's property `__contains_txn_messages` is when users try to upgrade. +- `__contains_txn_messages` is empty: broker initializes it by confirming whether there is TXN messages or not. +- **(Highlight)** `__contains_txn_messages` is not empty and is not typed `boolean`, rollback to the original behavior that always re-building `Two Indexes`, but you can never modify it anymore. +- **(Highlight)** `__contains_txn_messages` is not empty and is typed `boolean`, but it is a users' property, broker assumed that it is a system property. + +## Downgrade / Rollback + +You can downgrade or roll back gracefully. + +## Pulsar Geo-Replication Upgrade & Downgrade/Rollback Considerations + +The PIP does not affect Geo-Replication. + +# Links +* Mailing List discussion thread: https://lists.apache.org/thread/7mblhyvsrw5zybo0gs5512xg8f9sm67v +* Mailing List voting thread: https://lists.apache.org/thread/s6z5gcjyw081cxf9pwz361r8dt2k8gvl From d1753ee44221cd2bb9f16f18412617ab533112f0 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Fri, 29 Nov 2024 14:00:16 +0800 Subject: [PATCH 170/327] [fix][broker] Revert "[improve][client] Add log when can't add message to the container (#23657) --- .../java/org/apache/pulsar/client/impl/MessagesImpl.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java index 9768fd7c74b0f..d4cd36a22e15f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java @@ -22,13 +22,11 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import lombok.extern.slf4j.Slf4j; import net.jcip.annotations.NotThreadSafe; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Messages; @NotThreadSafe -@Slf4j public class MessagesImpl implements Messages { private final List> messageList; @@ -51,14 +49,10 @@ protected boolean canAdd(Message message) { return true; } if (maxNumberOfMessages > 0 && currentNumberOfMessages + 1 > maxNumberOfMessages) { - log.warn("can't add message to the container, has exceeded the maxNumberOfMessages : {} ", - maxNumberOfMessages); return false; } if (maxSizeOfMessages > 0 && currentSizeOfMessages + message.size() > maxSizeOfMessages) { - log.warn("can't add message to the container, has exceeded the maxSizeOfMessages : {} ", - maxSizeOfMessages); return false; } From eb60d0ab2022f11ebd2217a32ef60886f757cddf Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Fri, 29 Nov 2024 16:30:48 +0800 Subject: [PATCH 171/327] [fix][broker]: support missing broker level fine-granted permissions (#23637) --- .../authorization/AuthorizationProvider.java | 10 + .../authorization/AuthorizationService.java | 23 ++ .../pulsar/broker/admin/impl/BrokersBase.java | 100 ++++++- .../BrokerEndpointsAuthorizationTest.java | 277 ++++++++++++++++++ .../client/admin/internal/BrokersImpl.java | 2 +- .../common/policies/data/BrokerOperation.java | 39 +++ 6 files changed, 437 insertions(+), 14 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerEndpointsAuthorizationTest.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerOperation.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java index ffb38f770a9cc..48386265940a3 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java @@ -32,6 +32,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.BrokerOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -383,4 +384,13 @@ default CompletableFuture>> getPermissionsAsync(Name String.format("getPermissionsAsync on namespaceName %s is not supported by the Authorization", namespaceName))); } + + default CompletableFuture allowBrokerOperationAsync(String clusterName, + String brokerId, + BrokerOperation brokerOperation, + String role, + AuthenticationDataSource authData) { + return FutureUtil.failedFuture( + new UnsupportedOperationException("allowBrokerOperationAsync is not supported yet.")); + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java index 2951eb1f2973f..1348a405b0dfa 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java @@ -38,6 +38,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.BrokerOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -544,6 +545,28 @@ public CompletableFuture allowTenantOperationAsync(String tenantName, } } + public CompletableFuture allowBrokerOperationAsync(String clusterName, + String brokerId, + BrokerOperation brokerOperation, + String originalRole, + String role, + AuthenticationDataSource authData) { + if (!isValidOriginalPrincipal(role, originalRole, authData)) { + return CompletableFuture.completedFuture(false); + } + + if (isProxyRole(role)) { + final var isRoleAuthorizedFuture = provider.allowBrokerOperationAsync(clusterName, brokerId, + brokerOperation, role, authData); + final var isOriginalAuthorizedFuture = provider.allowBrokerOperationAsync(clusterName, brokerId, + brokerOperation, originalRole, authData); + return isRoleAuthorizedFuture.thenCombine(isOriginalAuthorizedFuture, + (isRoleAuthorized, isOriginalAuthorized) -> isRoleAuthorized && isOriginalAuthorized); + } else { + return provider.allowBrokerOperationAsync(clusterName, brokerId, brokerOperation, role, authData); + } + } + /** * @deprecated - will be removed after 2.12. Use async variant. */ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index e397dbb64a075..a24a78d8e3102 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -69,6 +69,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.policies.data.BrokerInfo; +import org.apache.pulsar.common.policies.data.BrokerOperation; import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ThreadDumpUtil; @@ -107,7 +108,8 @@ public class BrokersBase extends AdminResource { @ApiResponse(code = 404, message = "Cluster does not exist: cluster={clustername}") }) public void getActiveBrokers(@Suspended final AsyncResponse asyncResponse, @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(cluster == null ? pulsar().getConfiguration().getClusterName() + : cluster, pulsar().getBrokerId(), BrokerOperation.LIST_BROKERS) .thenCompose(__ -> validateClusterOwnershipAsync(cluster)) .thenCompose(__ -> pulsar().getLoadManager().get().getAvailableBrokersAsync()) .thenAccept(activeBrokers -> { @@ -148,7 +150,9 @@ public void getActiveBrokers(@Suspended final AsyncResponse asyncResponse) throw @ApiResponse(code = 403, message = "This operation requires super-user access"), @ApiResponse(code = 404, message = "Leader broker not found") }) public void getLeaderBroker(@Suspended final AsyncResponse asyncResponse) { - validateSuperUserAccessAsync().thenAccept(__ -> { + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), + pulsar().getBrokerId(), BrokerOperation.GET_LEADER_BROKER) + .thenAccept(__ -> { LeaderBroker leaderBroker = pulsar().getLeaderElectionService().getCurrentLeader() .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Couldn't find leader broker")); BrokerInfo brokerInfo = BrokerInfo.builder() @@ -175,7 +179,8 @@ public void getLeaderBroker(@Suspended final AsyncResponse asyncResponse) { public void getOwnedNamespaces(@Suspended final AsyncResponse asyncResponse, @PathParam("clusterName") String cluster, @PathParam("brokerId") String brokerId) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), + pulsar().getBrokerId(), BrokerOperation.LIST_OWNED_NAMESPACES) .thenCompose(__ -> maybeRedirectToBroker(brokerId)) .thenCompose(__ -> validateClusterOwnershipAsync(cluster)) .thenCompose(__ -> pulsar().getNamespaceService().getOwnedNameSpacesStatusAsync()) @@ -204,7 +209,8 @@ public void getOwnedNamespaces(@Suspended final AsyncResponse asyncResponse, public void updateDynamicConfiguration(@Suspended AsyncResponse asyncResponse, @PathParam("configName") String configName, @PathParam("configValue") String configValue) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.UPDATE_DYNAMIC_CONFIGURATION) .thenCompose(__ -> persistDynamicConfigurationAsync(configName, configValue)) .thenAccept(__ -> { LOG.info("[{}] Updated Service configuration {}/{}", clientAppId(), configName, configValue); @@ -228,7 +234,8 @@ public void updateDynamicConfiguration(@Suspended AsyncResponse asyncResponse, public void deleteDynamicConfiguration( @Suspended AsyncResponse asyncResponse, @PathParam("configName") String configName) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.DELETE_DYNAMIC_CONFIGURATION) .thenCompose(__ -> internalDeleteDynamicConfigurationOnMetadataAsync(configName)) .thenAccept(__ -> { LOG.info("[{}] Successfully to delete dynamic configuration {}", clientAppId(), configName); @@ -249,7 +256,8 @@ public void deleteDynamicConfiguration( @ApiResponse(code = 404, message = "Configuration not found"), @ApiResponse(code = 500, message = "Internal server error")}) public void getAllDynamicConfigurations(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.LIST_DYNAMIC_CONFIGURATIONS) .thenCompose(__ -> dynamicConfigurationResources().getDynamicConfigurationAsync()) .thenAccept(configOpt -> asyncResponse.resume(configOpt.orElseGet(Collections::emptyMap))) .exceptionally(ex -> { @@ -266,7 +274,8 @@ public void getAllDynamicConfigurations(@Suspended AsyncResponse asyncResponse) @ApiResponses(value = { @ApiResponse(code = 403, message = "You don't have admin permission to get configuration")}) public void getDynamicConfigurationName(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.LIST_DYNAMIC_CONFIGURATIONS) .thenAccept(__ -> asyncResponse.resume(pulsar().getBrokerService().getDynamicConfiguration())) .exceptionally(ex -> { LOG.error("[{}] Failed to get all dynamic configuration names.", clientAppId(), ex); @@ -281,7 +290,8 @@ public void getDynamicConfigurationName(@Suspended AsyncResponse asyncResponse) response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public void getRuntimeConfiguration(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.LIST_RUNTIME_CONFIGURATIONS) .thenAccept(__ -> asyncResponse.resume(pulsar().getBrokerService().getRuntimeConfiguration())) .exceptionally(ex -> { LOG.error("[{}] Failed to get runtime configuration.", clientAppId(), ex); @@ -322,7 +332,8 @@ private synchronized CompletableFuture persistDynamicConfigurationAsync( @ApiOperation(value = "Get the internal configuration data", response = InternalConfigurationData.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public void getInternalConfigurationData(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.GET_INTERNAL_CONFIGURATION_DATA) .thenAccept(__ -> asyncResponse.resume(pulsar().getInternalConfigurationData())) .exceptionally(ex -> { LOG.error("[{}] Failed to get internal configuration data.", clientAppId(), ex); @@ -339,7 +350,8 @@ public void getInternalConfigurationData(@Suspended AsyncResponse asyncResponse) @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 500, message = "Internal server error")}) public void backlogQuotaCheck(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.CHECK_BACKLOG_QUOTA) .thenAcceptAsync(__ -> { pulsar().getBrokerService().monitorBacklogQuota(); asyncResponse.resume(Response.noContent().build()); @@ -378,7 +390,8 @@ public void healthCheck(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "Topic Version") @QueryParam("topicVersion") TopicVersion topicVersion, @QueryParam("brokerId") String brokerId) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), StringUtils.isBlank(brokerId) + ? pulsar().getBrokerId() : brokerId, BrokerOperation.HEALTH_CHECK) .thenAccept(__ -> checkDeadlockedThreads()) .thenCompose(__ -> maybeRedirectToBroker( StringUtils.isBlank(brokerId) ? pulsar().getBrokerId() : brokerId)) @@ -596,8 +609,9 @@ public void shutDownBrokerGracefully( @QueryParam("forcedTerminateTopic") @DefaultValue("true") boolean forcedTerminateTopic, @Suspended final AsyncResponse asyncResponse ) { - validateSuperUserAccess(); - doShutDownBrokerGracefullyAsync(maxConcurrentUnloadPerSec, forcedTerminateTopic) + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.SHUTDOWN) + .thenCompose(__ -> doShutDownBrokerGracefullyAsync(maxConcurrentUnloadPerSec, forcedTerminateTopic)) .thenAccept(__ -> { LOG.info("[{}] Successfully shutdown broker gracefully", clientAppId()); asyncResponse.resume(Response.noContent().build()); @@ -614,5 +628,65 @@ private CompletableFuture doShutDownBrokerGracefullyAsync(int maxConcurren pulsar().getBrokerService().unloadNamespaceBundlesGracefully(maxConcurrentUnloadPerSec, forcedTerminateTopic); return pulsar().closeAsync(); } + + + private CompletableFuture validateBothSuperuserAndBrokerOperation(String cluster, String brokerId, + BrokerOperation operation) { + final var superUserAccessValidation = validateSuperUserAccessAsync(); + final var brokerOperationValidation = validateBrokerOperationAsync(cluster, brokerId, operation); + return FutureUtil.waitForAll(List.of(superUserAccessValidation, brokerOperationValidation)) + .handle((result, err) -> { + if (!superUserAccessValidation.isCompletedExceptionally() + || !brokerOperationValidation.isCompletedExceptionally()) { + return null; + } + if (LOG.isDebugEnabled()) { + Throwable superUserValidationException = null; + try { + superUserAccessValidation.join(); + } catch (Throwable ex) { + superUserValidationException = FutureUtil.unwrapCompletionException(ex); + } + Throwable brokerOperationValidationException = null; + try { + brokerOperationValidation.join(); + } catch (Throwable ex) { + brokerOperationValidationException = FutureUtil.unwrapCompletionException(ex); + } + LOG.debug("validateBothSuperuserAndBrokerOperation failed." + + " originalPrincipal={} clientAppId={} operation={} broker={} " + + "superuserValidationError={} brokerOperationValidationError={}", + originalPrincipal(), clientAppId(), operation.toString(), brokerId, + superUserValidationException, brokerOperationValidationException); + } + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBothSuperuserAndBrokerOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on broker [%s]", + originalPrincipal(), clientAppId(), operation.toString(), brokerId)); + }); + } + + + private CompletableFuture validateBrokerOperationAsync(String cluster, String brokerId, + BrokerOperation operation) { + final var pulsar = pulsar(); + if (pulsar.getBrokerService().isAuthenticationEnabled() + && pulsar.getBrokerService().isAuthorizationEnabled()) { + return pulsar.getBrokerService().getAuthorizationService() + .allowBrokerOperationAsync(cluster, brokerId, operation, originalPrincipal(), + clientAppId(), clientAuthData()) + .thenAccept(isAuthorized -> { + if (!isAuthorized) { + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBrokerOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on broker [%s]", + originalPrincipal(), clientAppId(), operation.toString(), brokerId)); + } + }); + } + return CompletableFuture.completedFuture(null); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerEndpointsAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerEndpointsAuthorizationTest.java new file mode 100644 index 0000000000000..ef66f005b3c58 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerEndpointsAuthorizationTest.java @@ -0,0 +1,277 @@ +/* + * 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.admin; + + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.naming.TopicVersion; +import org.apache.pulsar.common.policies.data.BrokerOperation; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +public class BrokerEndpointsAuthorizationTest extends MockedPulsarStandalone { + private AuthorizationService orignalAuthorizationService; + private AuthorizationService spyAuthorizationService; + + private PulsarAdmin superUserAdmin; + private PulsarAdmin nobodyAdmin; + + @SneakyThrows + @BeforeClass(alwaysRun = true) + public void setup() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + start(); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + this.nobodyAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(NOBODY_TOKEN)) + .build(); + } + + @BeforeMethod(alwaysRun = true) + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + spyAuthorizationService = spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + spyAuthorizationService, true); + } + + @AfterMethod(alwaysRun = true) + public void after() throws IllegalAccessException { + if (orignalAuthorizationService != null) { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", orignalAuthorizationService, true); + } + } + + @SneakyThrows + @AfterClass(alwaysRun = true) + public void cleanup() { + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + spyAuthorizationService = null; + orignalAuthorizationService = null; + super.close(); + } + + @Test + public void testGetActiveBroker() throws PulsarAdminException { + superUserAdmin.brokers().getActiveBrokers(); + final String brokerId = getPulsarService().getBrokerId(); + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_BROKERS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getActiveBrokers()); + } + + @Test + public void testGetActiveBrokerWithCluster() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.brokers().getActiveBrokers(clusterName); + final String brokerId = getPulsarService().getBrokerId(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_BROKERS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getActiveBrokers(clusterName)); + } + + @Test + public void testGetLeaderBroker() throws PulsarAdminException { + superUserAdmin.brokers().getLeaderBroker(); + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.GET_LEADER_BROKER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getLeaderBroker()); + } + + @Test + public void testGetOwnedNamespaces() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getOwnedNamespaces(clusterName, brokerId); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_OWNED_NAMESPACES), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getOwnedNamespaces(clusterName, brokerId)); + } + + @Test + public void testUpdateDynamicConfiguration() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().updateDynamicConfiguration("maxTenants", "10"); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.UPDATE_DYNAMIC_CONFIGURATION), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().updateDynamicConfiguration("maxTenants", "10")); + } + + @Test + public void testDeleteDynamicConfiguration() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().deleteDynamicConfiguration("maxTenants"); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.DELETE_DYNAMIC_CONFIGURATION), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().deleteDynamicConfiguration("maxTenants")); + } + + + @Test + public void testGetAllDynamicConfiguration() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getAllDynamicConfigurations(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_DYNAMIC_CONFIGURATIONS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getAllDynamicConfigurations()); + } + + + @Test + public void testGetDynamicConfigurationName() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getDynamicConfigurationNames(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_DYNAMIC_CONFIGURATIONS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getDynamicConfigurationNames()); + } + + + @Test + public void testGetRuntimeConfiguration() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getRuntimeConfigurations(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_RUNTIME_CONFIGURATIONS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getRuntimeConfigurations()); + } + + + @Test + public void testGetInternalConfigurationData() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getInternalConfigurationData(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.GET_INTERNAL_CONFIGURATION_DATA), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getInternalConfigurationData()); + } + + + @Test + public void testBacklogQuotaCheck() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().backlogQuotaCheck(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.CHECK_BACKLOG_QUOTA), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().backlogQuotaCheck()); + } + + @Test + public void testHealthCheck() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().healthcheck(TopicVersion.V2); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.HEALTH_CHECK), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().healthcheck(TopicVersion.V2)); + } +} diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java index 35b261b196eee..b0cd3edeb21fe 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java @@ -162,7 +162,7 @@ public void backlogQuotaCheck() throws PulsarAdminException { @Override public CompletableFuture backlogQuotaCheckAsync() { - WebTarget path = adminBrokers.path("backlogQuotaCheck"); + WebTarget path = adminBrokers.path("backlog-quota-check"); return asyncGetRequest(path, new FutureCallback() {}); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerOperation.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerOperation.java new file mode 100644 index 0000000000000..de053fea6ad4a --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerOperation.java @@ -0,0 +1,39 @@ +/* + * 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.common.policies.data; + +public enum BrokerOperation { + LIST_BROKERS, + GET_BROKER, + + GET_LEADER_BROKER, + LIST_OWNED_NAMESPACES, + + LIST_DYNAMIC_CONFIGURATIONS, + UPDATE_DYNAMIC_CONFIGURATION, + DELETE_DYNAMIC_CONFIGURATION, + + LIST_RUNTIME_CONFIGURATIONS, + + GET_INTERNAL_CONFIGURATION_DATA, + + CHECK_BACKLOG_QUOTA, + HEALTH_CHECK, + SHUTDOWN +} From 9ed44dd77f81abeb63fd5ed2877a1601afdc0bcc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 29 Nov 2024 11:05:29 +0200 Subject: [PATCH 172/327] [improve] Upgrade OpenTelemetry library to 1.44.1 version (#23656) --- .../server/src/assemble/LICENSE.bin.txt | 41 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +-- pom.xml | 2 +- .../stats/BrokerOpenTelemetryTestUtil.java | 13 ++++-- .../client/metrics/ClientMetricsTest.java | 18 ++++++-- 5 files changed, 47 insertions(+), 33 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index fd393cfec9b76..10899bc0ae7aa 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -338,12 +338,11 @@ The Apache Software License, Version 2.0 - io.prometheus-simpleclient_tracer_otel-0.16.0.jar - io.prometheus-simpleclient_tracer_otel_agent-0.16.0.jar * Prometheus exporter - - io.prometheus-prometheus-metrics-config-1.3.1.jar - - io.prometheus-prometheus-metrics-exporter-common-1.3.1.jar - - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.1.jar - - io.prometheus-prometheus-metrics-exposition-formats-1.3.1.jar - - io.prometheus-prometheus-metrics-model-1.3.1.jar - - io.prometheus-prometheus-metrics-shaded-protobuf-1.3.1.jar + - io.prometheus-prometheus-metrics-config-1.3.3.jar + - io.prometheus-prometheus-metrics-exporter-common-1.3.3.jar + - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.3.jar + - io.prometheus-prometheus-metrics-exposition-formats-1.3.3.jar + - io.prometheus-prometheus-metrics-model-1.3.3.jar * Jakarta Bean Validation API - jakarta.validation-jakarta.validation-api-2.0.2.jar - javax.validation-validation-api-1.1.0.Final.jar @@ -516,21 +515,21 @@ The Apache Software License, Version 2.0 * RoaringBitmap - org.roaringbitmap-RoaringBitmap-1.2.0.jar * OpenTelemetry - - io.opentelemetry-opentelemetry-api-1.41.0.jar - - io.opentelemetry-opentelemetry-api-incubator-1.41.0-alpha.jar - - io.opentelemetry-opentelemetry-context-1.41.0.jar - - io.opentelemetry-opentelemetry-exporter-common-1.41.0.jar - - io.opentelemetry-opentelemetry-exporter-otlp-1.41.0.jar - - io.opentelemetry-opentelemetry-exporter-otlp-common-1.41.0.jar - - io.opentelemetry-opentelemetry-exporter-prometheus-1.41.0-alpha.jar - - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-common-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-logs-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-metrics-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-trace-1.41.0.jar + - io.opentelemetry-opentelemetry-api-1.44.1.jar + - io.opentelemetry-opentelemetry-api-incubator-1.44.1-alpha.jar + - io.opentelemetry-opentelemetry-context-1.44.1.jar + - io.opentelemetry-opentelemetry-exporter-common-1.44.1.jar + - io.opentelemetry-opentelemetry-exporter-otlp-1.44.1.jar + - io.opentelemetry-opentelemetry-exporter-otlp-common-1.44.1.jar + - io.opentelemetry-opentelemetry-exporter-prometheus-1.44.1-alpha.jar + - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-common-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-logs-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-metrics-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-trace-1.44.1.jar - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.33.6.jar - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.33.6-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-resources-1.33.6-alpha.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 1601f32bb2b34..07a40d3bc1bc8 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -388,9 +388,9 @@ The Apache Software License, Version 2.0 - log4j-slf4j2-impl-2.23.1.jar - log4j-web-2.23.1.jar * OpenTelemetry - - opentelemetry-api-1.41.0.jar - - opentelemetry-api-incubator-1.41.0-alpha.jar - - opentelemetry-context-1.41.0.jar + - opentelemetry-api-1.44.1.jar + - opentelemetry-api-incubator-1.44.1-alpha.jar + - opentelemetry-context-1.44.1.jar * BookKeeper - bookkeeper-common-allocator-4.17.1.jar diff --git a/pom.xml b/pom.xml index e6d154c1b34d6..4830358f5ea72 100644 --- a/pom.xml +++ b/pom.xml @@ -258,7 +258,7 @@ flexible messaging model and an intuitive client API. 3.4.3 1.5.2-3 2.0.6 - 1.41.0 + 1.44.1 ${opentelemetry.version}-alpha 1.33.6 ${opentelemetry.instrumentation.version}-alpha diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java index 0d46e80a70302..3bfbf2064e156 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.stats; import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; +import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.metrics.data.MetricData; @@ -52,13 +53,14 @@ public static void disableExporters(AutoConfiguredOpenTelemetrySdkBuilder sdkBui public static void assertMetricDoubleSumValue(Collection metrics, String metricName, Attributes attributes, Consumer valueConsumer) { + Map, Object> attributesMap = attributes.asMap(); assertThat(metrics) .anySatisfy(metric -> assertThat(metric) .hasName(metricName) .hasDoubleSumSatisfying(sum -> sum.satisfies( sumData -> assertThat(sumData.getPoints()).anySatisfy( point -> { - assertThat(point.getAttributes()).isEqualTo(attributes); + assertThat(point.getAttributes().asMap()).isEqualTo(attributesMap); valueConsumer.accept(point.getValue()); })))); } @@ -70,13 +72,14 @@ public static void assertMetricLongSumValue(Collection metrics, Stri public static void assertMetricLongSumValue(Collection metrics, String metricName, Attributes attributes, Consumer valueConsumer) { + Map, Object> attributesMap = attributes.asMap(); assertThat(metrics) .anySatisfy(metric -> assertThat(metric) .hasName(metricName) .hasLongSumSatisfying(sum -> sum.satisfies( sumData -> assertThat(sumData.getPoints()).anySatisfy( point -> { - assertThat(point.getAttributes()).isEqualTo(attributes); + assertThat(point.getAttributes().asMap()).isEqualTo(attributesMap); valueConsumer.accept(point.getValue()); })))); } @@ -88,13 +91,14 @@ public static void assertMetricLongGaugeValue(Collection metrics, St public static void assertMetricLongGaugeValue(Collection metrics, String metricName, Attributes attributes, Consumer valueConsumer) { + Map, Object> attributesMap = attributes.asMap(); assertThat(metrics) .anySatisfy(metric -> assertThat(metric) .hasName(metricName) .hasLongGaugeSatisfying(gauge -> gauge.satisfies( pointData -> assertThat(pointData.getPoints()).anySatisfy( point -> { - assertThat(point.getAttributes()).isEqualTo(attributes); + assertThat(point.getAttributes().asMap()).isEqualTo(attributesMap); valueConsumer.accept(point.getValue()); })))); } @@ -106,13 +110,14 @@ public static void assertMetricDoubleGaugeValue(Collection metrics, public static void assertMetricDoubleGaugeValue(Collection metrics, String metricName, Attributes attributes, Consumer valueConsumer) { + Map, Object> attributesMap = attributes.asMap(); assertThat(metrics) .anySatisfy(metric -> assertThat(metric) .hasName(metricName) .hasDoubleGaugeSatisfying(gauge -> gauge.satisfies( pointData -> assertThat(pointData.getPoints()).anySatisfy( point -> { - assertThat(point.getAttributes()).isEqualTo(attributes); + assertThat(point.getAttributes().asMap()).isEqualTo(attributesMap); valueConsumer.accept(point.getValue()); })))); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java index 31305123c4148..02b38acf865d4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java @@ -21,7 +21,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.fail; -import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.metrics.SdkMeterProvider; @@ -48,7 +48,7 @@ public class ClientMetricsTest extends ProducerConsumerBase { InMemoryMetricReader reader; - OpenTelemetry otel; + OpenTelemetrySdk otel; @BeforeMethod @Override @@ -67,6 +67,14 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { super.internalCleanup(); + if (otel != null) { + otel.close(); + otel = null; + } + if (reader != null) { + reader.close(); + reader = null; + } } private Map collectMetrics() { @@ -88,8 +96,9 @@ private long getCounterValue(Map metrics, String name, assertNotNull(md, "metric not found: " + name); assertEquals(md.getType(), MetricDataType.LONG_SUM); + Map, Object> expectedAttributesMap = expectedAttributes.asMap(); for (var ex : md.getLongSumData().getPoints()) { - if (ex.getAttributes().equals(expectedAttributes)) { + if (ex.getAttributes().asMap().equals(expectedAttributesMap)) { return ex.getValue(); } } @@ -109,8 +118,9 @@ private long getHistoCountValue(Map metrics, String name, assertNotNull(md, "metric not found: " + name); assertEquals(md.getType(), MetricDataType.HISTOGRAM); + Map, Object> expectedAttributesMap = expectedAttributes.asMap(); for (var ex : md.getHistogramData().getPoints()) { - if (ex.getAttributes().equals(expectedAttributes)) { + if (ex.getAttributes().asMap().equals(expectedAttributesMap)) { return ex.getCount(); } } From 429f7de96826d20bc90eb54a3b40df0e31f4b42f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 29 Nov 2024 11:17:08 +0200 Subject: [PATCH 173/327] [improve][client] Reduce unshaded dependencies and shading warnings in shaded Java client modules (#23647) --- .../server/src/assemble/LICENSE.bin.txt | 2 - .../shell/src/assemble/LICENSE.bin.txt | 3 - jetcd-core-shaded/pom.xml | 4 + pom.xml | 30 +- pulsar-broker-auth-oidc/pom.xml | 4 + pulsar-broker-common/pom.xml | 4 +- pulsar-broker/pom.xml | 4 +- pulsar-client-admin-shaded/pom.xml | 311 ++++++++++------- ...sar.shade.javax.ws.rs.client.ClientBuilder | 1 + ...lsar.shade.javax.ws.rs.ext.RuntimeDelegate | 1 + pulsar-client-admin/pom.xml | 4 +- pulsar-client-all/pom.xml | 312 ++++++++++-------- ...sar.shade.javax.ws.rs.client.ClientBuilder | 1 + ...lsar.shade.javax.ws.rs.ext.RuntimeDelegate | 1 + pulsar-client-auth-athenz/pom.xml | 5 + pulsar-client-auth-sasl/pom.xml | 4 +- pulsar-client-shaded/pom.xml | 240 ++++++++------ pulsar-common/pom.xml | 4 +- pulsar-functions/proto/pom.xml | 6 +- pulsar-functions/runtime/pom.xml | 6 +- pulsar-functions/secrets/pom.xml | 6 +- pulsar-proxy/pom.xml | 4 +- tiered-storage/jcloud/pom.xml | 4 +- 23 files changed, 580 insertions(+), 381 deletions(-) create mode 100644 pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder create mode 100644 pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate create mode 100644 pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder create mode 100644 pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 10899bc0ae7aa..21422a41dcef9 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -571,10 +571,8 @@ Protocol Buffers License CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - com.sun.activation-javax.activation-1.2.0.jar - - javax.xml.bind-jaxb-api-2.3.1.jar * Java Servlet API -- javax.servlet-javax.servlet-api-3.1.0.jar * WebSocket Server API -- javax.websocket-javax.websocket-client-api-1.0.jar - * Java Web Service REST API -- javax.ws.rs-javax.ws.rs-api-2.1.jar * HK2 - Dependency Injection Kernel - org.glassfish.hk2-hk2-api-2.6.1.jar - org.glassfish.hk2-hk2-locator-2.6.1.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 07a40d3bc1bc8..a21c272f91b1d 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -431,11 +431,8 @@ MIT License CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - - javax.annotation-api-1.3.2.jar - javax.activation-1.2.0.jar - - jaxb-api-2.3.1.jar * WebSocket Server API -- javax.websocket-client-api-1.0.jar - * Java Web Service REST API -- javax.ws.rs-api-2.1.jar * HK2 - Dependency Injection Kernel - hk2-api-2.6.1.jar - hk2-locator-2.6.1.jar diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index 0b79df8278f81..2a5536987cd42 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -45,6 +45,10 @@ io.netty * + + javax.annotation + javax.annotation-api + diff --git a/pom.xml b/pom.xml index 4830358f5ea72..54744a253bb2b 100644 --- a/pom.xml +++ b/pom.xml @@ -220,7 +220,7 @@ flexible messaging model and an intuitive client API. 1.10 2.14.0 1.15 - 2.1 + 2.1.6 2.1.9 3.1.0 2.9.1 @@ -231,7 +231,7 @@ flexible messaging model and an intuitive client API. 2.1.0 3.24.2 1.18.32 - 1.3.2 + 1.3.5 2.3.1 1.2.0 1.2.2 @@ -854,9 +854,9 @@ flexible messaging model and an intuitive client API. - javax.ws.rs - javax.ws.rs-api - ${javax.ws.rs-api.version} + jakarta.ws.rs + jakarta.ws.rs-api + ${jakarta.ws.rs-api.version} @@ -990,6 +990,12 @@ flexible messaging model and an intuitive client API. com.yahoo.athenz athenz-zts-java-client-core ${athenz.version} + + + javax.ws.rs + javax.ws.rs-api + + @@ -1418,9 +1424,9 @@ flexible messaging model and an intuitive client API. - javax.annotation - javax.annotation-api - ${javax.annotation-api.version} + jakarta.annotation + jakarta.annotation-api + ${jakarta.annotation-api.version} @@ -1723,11 +1729,6 @@ flexible messaging model and an intuitive client API. lombok provided - - javax.annotation - javax.annotation-api - provided - @@ -2003,6 +2004,9 @@ flexible messaging model and an intuitive client API. src/assemble/LICENSE.bin.txt src/assemble/NOTICE.bin.txt + + **/META-INF/services/* + src/main/java/org/apache/bookkeeper/mledger/proto/MLDataFormats.java diff --git a/pulsar-broker-auth-oidc/pom.xml b/pulsar-broker-auth-oidc/pom.xml index 72351bf47d288..2d0931c3f2dfe 100644 --- a/pulsar-broker-auth-oidc/pom.xml +++ b/pulsar-broker-auth-oidc/pom.xml @@ -95,6 +95,10 @@ bcprov-jdk18on org.bouncycastle + + javax.annotation + javax.annotation-api + diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index 30b703cd78a92..858a234a32dcd 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -60,8 +60,8 @@ - javax.ws.rs - javax.ws.rs-api + jakarta.ws.rs + jakarta.ws.rs-api diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 97ede1f76e969..a34f566447006 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -446,8 +446,8 @@ - javax.xml.bind - jaxb-api + jakarta.xml.bind + jakarta.xml.bind-api javax.activation diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index ab42f0e2aef59..74ced063fbfd4 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -1,4 +1,4 @@ - + - + 4.0.0 org.apache.pulsar pulsar 4.1.0-SNAPSHOT - pulsar-client-admin Pulsar Client Admin - ${project.groupId} @@ -73,7 +70,6 @@ - maven-antrun-plugin @@ -86,15 +82,13 @@ - + - org.apache.maven.plugins maven-shade-plugin @@ -107,48 +101,55 @@ true true - - org.apache.pulsar:pulsar-client-original - org.apache.pulsar:pulsar-client-admin-original - org.apache.commons:commons-lang3 - commons-codec:commons-codec - commons-collections:commons-collections - org.asynchttpclient:* - org.reactivestreams:reactive-streams - com.typesafe.netty:netty-reactive-streams - org.javassist:javassist - com.google.guava:guava + com.fasterxml.jackson.*:* + com.google.*:* + com.google.auth:* com.google.code.gson:gson + com.google.guava:guava com.google.re2j:re2j com.spotify:completable-futures - com.fasterxml.jackson.*:* - io.netty:* - io.netty.incubator:* - org.apache.pulsar:pulsar-common - org.apache.bookkeeper:* + com.squareup.*:* + com.sun.activation:javax.activation + com.typesafe.netty:netty-reactive-streams + com.yahoo.datasketches:* com.yahoo.datasketches:sketches-core - org.glassfish.jersey*:* - javax.ws.rs:* - javax.xml.bind:jaxb-api - jakarta.annotation:* - org.glassfish.hk2*:* + commons-*:* + commons-codec:commons-codec + commons-collections:commons-collections + io.airlift:* io.grpc:* + io.netty.incubator:* + io.netty:* + io.opencensus:* io.perfmark:* - com.yahoo.datasketches:* - com.squareup.*:* - com.google.*:* - commons-*:* + io.swagger:* + jakarta.activation:jakarta.activation-api + jakarta.annotation:jakarta.annotation-api + jakarta.ws.rs:jakarta.ws.rs-api + jakarta.xml.bind:jakarta.xml.bind-api + javax.ws.rs:* + javax.xml.bind:jaxb-api + net.jcip:jcip-annotations + org.apache.bookkeeper:* + org.apache.commons:commons-compress + org.apache.commons:commons-lang3 + org.apache.pulsar:pulsar-client-admin-original + + org.apache.pulsar:pulsar-client-messagecrypto-bc + org.apache.pulsar:pulsar-client-original + org.apache.pulsar:pulsar-common + org.asynchttpclient:* + org.checkerframework:* org.eclipse.jetty:* - com.google.auth:* + org.glassfish.hk2*:* + org.glassfish.jersey*:* + org.javassist:javassist org.jvnet.mimepull:* - io.opencensus:* org.objenesis:* + org.reactivestreams:reactive-streams org.yaml:snakeyaml - io.swagger:* - - org.apache.pulsar:pulsar-client-messagecrypto-bc com.fasterxml.jackson.core:jackson-annotations @@ -162,7 +163,7 @@ ** - + org/bouncycastle/** @@ -172,19 +173,54 @@ ** - + org/bouncycastle/** + + org.asynchttpclient:async-http-client + + ** + + + org/asynchttpclient/config/ahc-default.properties + + + + *:* + + **/module-info.class + findbugsExclude.xml + META-INF/*-LICENSE + META-INF/*-NOTICE + META-INF/*.DSA + META-INF/*.RSA + META-INF/*.SF + META-INF/DEPENDENCIES* + META-INF/io.netty.versions.properties + META-INF/LICENSE* + META-INF/license/** + META-INF/MANIFEST.MF + META-INF/maven/** + META-INF/native-image/** + META-INF/NOTICE* + META-INF/proguard/** + + - - org.asynchttpclient - org.apache.pulsar.shade.org.asynchttpclient + + + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true - org.apache.commons - org.apache.pulsar.shade.org.apache.commons + com.fasterxml.jackson + org.apache.pulsar.shade.com.fasterxml.jackson + + com.fasterxml.jackson.annotation.* + com.google @@ -198,44 +234,61 @@ org.apache.pulsar.shade.com.spotify.futures - com.fasterxml.jackson - org.apache.pulsar.shade.com.fasterxml.jackson - - com.fasterxml.jackson.annotation.* - + com.squareup + org.apache.pulsar.shade.com.squareup - io.netty - org.apache.pulsar.shade.io.netty + com.sun.activation + org.apache.pulsar.shade.com.sun.activation - org.apache.pulsar.policies - org.apache.pulsar.shade.org.apache.pulsar.policies - - - org.apache.pulsar.checksum - org.apache.pulsar.shade.org.apache.pulsar.checksum + com.typesafe + org.apache.pulsar.shade.com.typesafe com.yahoo org.apache.pulsar.shade.com.yahoo - com.typesafe - org.apache.pulsar.shade.com.typesafe + io.airlift + org.apache.pulsar.shade.io.airlift - org.glassfish - org.apache.pulsar.shade.org.glassfish + io.grpc + org.apache.pulsar.shade.io.grpc - javax.ws - org.apache.pulsar.shade.javax.ws + io.netty + org.apache.pulsar.shade.io.netty + + + io.opencensus + org.apache.pulsar.shade.io.opencensus + + + io.swagger + org.apache.pulsar.shade.io.swagger + + + javassist + org.apache.pulsar.shade.javassist + + + javax.activation + org.apache.pulsar.shade.javax.activation javax.annotation org.apache.pulsar.shade.javax.annotation + + javax.inject + org.apache.pulsar.shade.javax.inject + + + javax.ws + org.apache.pulsar.shade.javax.ws + javax.xml.bind org.apache.pulsar.shade.javax.xml.bind @@ -245,71 +298,95 @@ org.apache.pulsar.shade.jersey - org.jvnet - org.apache.pulsar.shade.org.jvnet + META-INF/versions/(\d+)/com/fasterxml/jackson/core/ + META-INF/versions/$1/org/apache/pulsar/shade/com/fasterxml/jackson/core/ + + true + + + META-INF/versions/(\d+)/javax/xml/bind/ + META-INF/versions/$1/org/apache/pulsar/shade/javax/xml/bind/ + true + + + META-INF/versions/(\d+)/org/glassfish/ + META-INF/versions/$1/org/apache/pulsar/shade/org/glassfish/ + true + + + META-INF/versions/(\d+)/org/yaml/ + META-INF/versions/$1/org/apache/pulsar/shade/org/yaml/ + true + + + net.jcip + org.apache.pulsar.shade.net.jcip + + + okio + org.apache.pulsar.shade.okio org.aopalliance org.apache.pulsar.shade.org.aopalliance - javassist - org.apache.pulsar.shade.javassist + org.apache.bookkeeper + org.apache.pulsar.shade.org.apache.bookkeeper - javax.inject - org.apache.pulsar.shade.javax.inject + org.apache.commons + org.apache.pulsar.shade.org.apache.commons + + + org.apache.pulsar.checksum + org.apache.pulsar.shade.org.apache.pulsar.checksum + + + org.apache.pulsar.policies + org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.asynchttpclient + org.apache.pulsar.shade.org.asynchttpclient + + + org.checkerframework + org.apache.pulsar.shade.org.checkerframework + + + org.eclipse.jetty + org.apache.pulsar.shade.org.eclipse.jetty + + + org.glassfish + org.apache.pulsar.shade.org.glassfish + + + org.jvnet + org.apache.pulsar.shade.org.jvnet + + + org.objenesis + org.apache.pulsar.shade.org.objenesis org.reactivestreams org.apache.pulsar.shade.org.reactivestreams - - io.grpc - org.apache.pulsar.shade.io.grpc - - - okio - org.apache.pulsar.shade.okio - - - com.squareup - org.apache.pulsar.shade.com.squareup - - - io.opencensus - org.apache.pulsar.shade.io.opencensus - - - org.eclipse.jetty - org.apache.pulsar.shade.org.eclipse.jetty - - - org.objenesis - org.apache.pulsar.shade.org.objenesis - - - org.yaml - org.apache.pulsar.shade.org.yaml - - - io.swagger - org.apache.pulsar.shade.io.swagger - - - org.apache.bookkeeper - org.apache.pulsar.shade.org.apache.bookkeeper - - - - (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ - $1org_apache_pulsar_shade_$3 - true - + + org.yaml + org.apache.pulsar.shade.org.yaml + - - + + + + + true + + diff --git a/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder b/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder new file mode 100644 index 0000000000000..99a08cc8ca4d1 --- /dev/null +++ b/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder @@ -0,0 +1 @@ +org.apache.pulsar.shade.org.glassfish.jersey.client.JerseyClientBuilder \ No newline at end of file diff --git a/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate b/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate new file mode 100644 index 0000000000000..0adc919b7f0c2 --- /dev/null +++ b/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate @@ -0,0 +1 @@ +org.apache.pulsar.shade.org.glassfish.jersey.internal.RuntimeDelegateImpl \ No newline at end of file diff --git a/pulsar-client-admin/pom.xml b/pulsar-client-admin/pom.xml index 36070618ed891..0a94e48e9b939 100644 --- a/pulsar-client-admin/pom.xml +++ b/pulsar-client-admin/pom.xml @@ -76,8 +76,8 @@ - javax.xml.bind - jaxb-api + jakarta.xml.bind + jakarta.xml.bind-api javax.activation diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index e26f6eeac57bf..74007745c70ee 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -1,3 +1,4 @@ + + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.apache.pulsar pulsar 4.1.0-SNAPSHOT - pulsar-client-all Pulsar Client All - ${project.groupId} @@ -69,7 +67,6 @@ test - @@ -107,7 +104,6 @@ - maven-antrun-plugin @@ -120,15 +116,13 @@ - + - org.apache.maven.plugins @@ -145,67 +139,67 @@ true true false - - org.apache.pulsar:pulsar-client-original - org.apache.pulsar:pulsar-client-admin-original - org.apache.commons:commons-lang3 - commons-codec:commons-codec - commons-collections:commons-collections - org.asynchttpclient:* - io.netty:netty-codec-http - io.netty:netty-transport-native-epoll - org.reactivestreams:reactive-streams - com.typesafe.netty:netty-reactive-streams - org.javassist:javassist - com.google.guava:* - org.checkerframework:* + com.fasterxml.jackson.*:* + com.google.*:* + com.google.auth:* com.google.code.findbugs:* + com.google.code.gson:gson com.google.errorprone:* + com.google.guava:* com.google.j2objc:* - com.google.code.gson:gson com.google.re2j:re2j com.spotify:completable-futures - com.fasterxml.jackson.*:* - io.netty:netty - io.netty:netty-all - io.netty:netty-tcnative-boringssl-static - org.eclipse.jetty:* + com.squareup.*:* + com.sun.activation:javax.activation + + com.thoughtworks.paranamer:paranamer + com.typesafe.netty:netty-reactive-streams com.yahoo.datasketches:* + com.yahoo.datasketches:sketches-core commons-*:* - io.swagger:* + commons-codec:commons-codec + commons-collections:commons-collections io.airlift:* - - org.apache.pulsar:pulsar-common - org.apache.bookkeeper:* - com.yahoo.datasketches:sketches-core - org.glassfish.jersey*:* - javax.ws.rs:* - javax.xml.bind:jaxb-api - jakarta.annotation:* - org.glassfish.hk2*:* io.grpc:* - io.perfmark:* - com.yahoo.datasketches:* - io.netty:* io.netty.incubator:* - com.squareup.*:* - com.google.*:* - commons-*:* - org.eclipse.jetty:* - com.google.auth:* - org.jvnet.mimepull:* + io.netty:* + io.netty:netty + io.netty:netty-all + io.netty:netty-codec-http + io.netty:netty-tcnative-boringssl-static + io.netty:netty-transport-native-epoll io.opencensus:* - org.objenesis:* - org.yaml:snakeyaml + io.perfmark:* + io.swagger:* + jakarta.activation:jakarta.activation-api + jakarta.annotation:jakarta.annotation-api + jakarta.ws.rs:jakarta.ws.rs-api + jakarta.xml.bind:jakarta.xml.bind-api + javax.ws.rs:* + javax.xml.bind:jaxb-api + net.jcip:jcip-annotations org.apache.avro:* - - com.thoughtworks.paranamer:paranamer + org.apache.bookkeeper:* org.apache.commons:commons-compress - org.tukaani:xz + org.apache.commons:commons-lang3 + org.apache.pulsar:pulsar-client-admin-original org.apache.pulsar:pulsar-client-messagecrypto-bc + org.apache.pulsar:pulsar-client-original + org.apache.pulsar:pulsar-common + org.asynchttpclient:* + org.checkerframework:* + org.eclipse.jetty:* + org.glassfish.hk2*:* + org.glassfish.jersey*:* + org.javassist:javassist + org.jvnet.mimepull:* + org.objenesis:* + org.reactivestreams:reactive-streams + org.tukaani:xz + org.yaml:snakeyaml com.fasterxml.jackson.core:jackson-annotations @@ -219,23 +213,54 @@ ** - + org/bouncycastle/** + + org.asynchttpclient:async-http-client + + ** + + + org/asynchttpclient/config/ahc-default.properties + + + + *:* + + **/module-info.class + findbugsExclude.xml + META-INF/*-LICENSE + META-INF/*-NOTICE + META-INF/*.DSA + META-INF/*.RSA + META-INF/*.SF + META-INF/DEPENDENCIES* + META-INF/io.netty.versions.properties + META-INF/LICENSE* + META-INF/license/** + META-INF/MANIFEST.MF + META-INF/maven/** + META-INF/native-image/** + META-INF/NOTICE* + META-INF/proguard/** + + + - org.asynchttpclient - org.apache.pulsar.shade.org.asynchttpclient - - - org.apache.bookkeeper - org.apache.pulsar.shade.org.apache.bookkeeper + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true - org.apache.commons - org.apache.pulsar.shade.org.apache.commons + com.fasterxml.jackson + org.apache.pulsar.shade.com.fasterxml.jackson + + com.fasterxml.jackson.annotation.* + com.google @@ -249,103 +274,109 @@ org.apache.pulsar.shade.com.spotify.futures - com.fasterxml.jackson - org.apache.pulsar.shade.com.fasterxml.jackson - - com.fasterxml.jackson.annotation.* - + com.squareup + org.apache.pulsar.shade.com.squareup - io.netty - org.apache.pulsar.shade.io.netty + com.sun.activation + org.apache.pulsar.shade.com.sun.activation - org.apache.pulsar.policies - org.apache.pulsar.shade.org.apache.pulsar.policies + com.thoughtworks.paranamer + org.apache.pulsar.shade.com.thoughtworks.paranamer - com.yahoo.datasketches - org.apache.pulsar.shade.com.yahoo.datasketches + com.typesafe + org.apache.pulsar.shade.com.typesafe com.yahoo org.apache.pulsar.shade.com.yahoo - org.eclipse.jetty - org.apache.pulsar.shade.org.eclipse - - - org.reactivestreams - org.apache.pulsar.shade.org.reactivestreams + com.yahoo.datasketches + org.apache.pulsar.shade.com.yahoo.datasketches - com.typesafe - org.apache.pulsar.shade.com.typesafe + io.airlift + org.apache.pulsar.shade.io.airlift - javax.ws - org.apache.pulsar.shade.javax.ws + io.grpc + org.apache.pulsar.shade.io.grpc - javax.annotation - org.apache.pulsar.shade.javax.annotation + io.netty + org.apache.pulsar.shade.io.netty - javax.xml.bind - org.apache.pulsar.shade.javax.xml.bind + io.opencensus + org.apache.pulsar.shade.io.opencensus - jersey - org.apache.pulsar.shade.jersey + io.swagger + org.apache.pulsar.shade.io.swagger - org.jvnet - org.apache.pulsar.shade.org.jvnet + javassist + org.apache.pulsar.shade.javassist - org.aopalliance - org.apache.pulsar.shade.org.aopalliance + javax.activation + org.apache.pulsar.shade.javax.activation - javassist - org.apache.pulsar.shade.javassist + javax.annotation + org.apache.pulsar.shade.javax.annotation javax.inject org.apache.pulsar.shade.javax.inject - org.glassfish - org.apache.pulsar.shade.org.glassfish + javax.ws + org.apache.pulsar.shade.javax.ws + + + javax.xml.bind + org.apache.pulsar.shade.javax.xml.bind + + + jersey + org.apache.pulsar.shade.jersey - io.grpc - org.apache.pulsar.shade.io.grpc + META-INF/versions/(\d+)/com/fasterxml/jackson/core/ + META-INF/versions/$1/org/apache/pulsar/shade/com/fasterxml/jackson/core/ + + true - okio - org.apache.pulsar.shade.okio + META-INF/versions/(\d+)/javax/xml/bind/ + META-INF/versions/$1/org/apache/pulsar/shade/javax/xml/bind/ + true - com.squareup - org.apache.pulsar.shade.com.squareup + META-INF/versions/(\d+)/org/glassfish/ + META-INF/versions/$1/org/apache/pulsar/shade/org/glassfish/ + true - io.opencensus - org.apache.pulsar.shade.io.opencensus + META-INF/versions/(\d+)/org/yaml/ + META-INF/versions/$1/org/apache/pulsar/shade/org/yaml/ + true - org.eclipse.jetty - org.apache.pulsar.shade.org.eclipse.jetty + net.jcip + org.apache.pulsar.shade.net.jcip - org.objenesis - org.apache.pulsar.shade.org.objenesis + okio + org.apache.pulsar.shade.okio - org.yaml - org.apache.pulsar.shade.org.yaml + org.aopalliance + org.apache.pulsar.shade.org.aopalliance org.apache.avro @@ -363,45 +394,68 @@ org.apache.avro.reflect.Union - - - org.codehaus.jackson - org.apache.pulsar.shade.org.codehaus.jackson - - com.thoughtworks.paranamer - org.apache.pulsar.shade.com.thoughtworks.paranamer + org.apache.bookkeeper + org.apache.pulsar.shade.org.apache.bookkeeper org.apache.commons org.apache.pulsar.shade.org.apache.commons - io.airlift - org.apache.pulsar.shade.io.airlift + org.apache.pulsar.policies + org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.asynchttpclient + org.apache.pulsar.shade.org.asynchttpclient org.checkerframework org.apache.pulsar.shade.org.checkerframework + - javax.annotation - org.apache.pulsar.shade.javax.annotation + org.codehaus.jackson + org.apache.pulsar.shade.org.codehaus.jackson + + + org.eclipse.jetty + org.apache.pulsar.shade.org.eclipse + + + org.glassfish + org.apache.pulsar.shade.org.glassfish + + + org.jvnet + org.apache.pulsar.shade.org.jvnet + + + org.objenesis + org.apache.pulsar.shade.org.objenesis + + + org.reactivestreams + org.apache.pulsar.shade.org.reactivestreams org.tukaani org.apache.pulsar.shade.org.tukaani - - (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ - $1org_apache_pulsar_shade_$3 - true + org.yaml + org.apache.pulsar.shade.org.yaml - - + + + + + true + + diff --git a/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder b/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder new file mode 100644 index 0000000000000..99a08cc8ca4d1 --- /dev/null +++ b/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder @@ -0,0 +1 @@ +org.apache.pulsar.shade.org.glassfish.jersey.client.JerseyClientBuilder \ No newline at end of file diff --git a/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate b/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate new file mode 100644 index 0000000000000..0adc919b7f0c2 --- /dev/null +++ b/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate @@ -0,0 +1 @@ +org.apache.pulsar.shade.org.glassfish.jersey.internal.RuntimeDelegateImpl \ No newline at end of file diff --git a/pulsar-client-auth-athenz/pom.xml b/pulsar-client-auth-athenz/pom.xml index 99786b4d18dd8..0e994ce25c24e 100644 --- a/pulsar-client-auth-athenz/pom.xml +++ b/pulsar-client-auth-athenz/pom.xml @@ -46,6 +46,11 @@ athenz-zts-java-client-core + + jakarta.ws.rs + jakarta.ws.rs-api + + com.yahoo.athenz athenz-cert-refresher diff --git a/pulsar-client-auth-sasl/pom.xml b/pulsar-client-auth-sasl/pom.xml index 1a4720b3d589d..61d1157afda34 100644 --- a/pulsar-client-auth-sasl/pom.xml +++ b/pulsar-client-auth-sasl/pom.xml @@ -57,8 +57,8 @@ - javax.ws.rs - javax.ws.rs-api + jakarta.ws.rs + jakarta.ws.rs-api diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index cd87d901b21bb..1093b405731ea 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -1,3 +1,4 @@ + + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.apache.pulsar pulsar 4.1.0-SNAPSHOT - pulsar-client Pulsar Client Java - ${project.groupId} @@ -48,11 +46,8 @@ ${project.version} - - - org.apache.maven.plugins maven-dependency-plugin @@ -88,7 +83,6 @@ - maven-antrun-plugin @@ -101,15 +95,13 @@ - + - org.apache.maven.plugins @@ -124,50 +116,54 @@ true true false - - org.apache.pulsar:pulsar-client-original - org.apache.bookkeeper:* - org.apache.commons:commons-lang3 - commons-codec:commons-codec - commons-collections:commons-collections - org.asynchttpclient:* - io.netty:netty-codec-http - io.netty:netty-transport-native-epoll - org.reactivestreams:reactive-streams - com.typesafe.netty:netty-reactive-streams - org.javassist:javassist - com.google.guava:* - org.checkerframework:* + com.fasterxml.jackson.*:* com.google.code.findbugs:* + com.google.code.gson:gson com.google.errorprone:* + com.google.guava:* com.google.j2objc:* - com.google.code.gson:gson com.google.re2j:re2j com.spotify:completable-futures - com.fasterxml.jackson.*:* - io.netty:* - io.netty.incubator:* - io.perfmark:* - org.eclipse.jetty:* + com.sun.activation:javax.activation + + com.thoughtworks.paranamer:paranamer + com.typesafe.netty:netty-reactive-streams com.yahoo.datasketches:* + com.yahoo.datasketches:sketches-core commons-*:* - io.swagger:* + commons-codec:commons-codec + commons-collections:commons-collections io.airlift:* - - org.apache.pulsar:pulsar-common - com.yahoo.datasketches:sketches-core - org.objenesis:* - org.yaml:snakeyaml - + io.netty.incubator:* + io.netty:* + io.netty:netty-codec-http + io.netty:netty-transport-native-epoll + io.perfmark:* + io.swagger:* + jakarta.activation:jakarta.activation-api + jakarta.annotation:jakarta.annotation-api + jakarta.ws.rs:jakarta.ws.rs-api + jakarta.xml.bind:jakarta.xml.bind-api + javax.ws.rs:* + net.jcip:jcip-annotations org.apache.avro:* - - com.thoughtworks.paranamer:paranamer + org.apache.bookkeeper:* org.apache.commons:commons-compress - org.tukaani:xz + org.apache.commons:commons-lang3 org.apache.pulsar:pulsar-client-messagecrypto-bc + org.apache.pulsar:pulsar-client-original + org.apache.pulsar:pulsar-common + org.asynchttpclient:* + org.checkerframework:* + org.eclipse.jetty:* + org.javassist:javassist + org.objenesis:* + org.reactivestreams:reactive-streams + org.tukaani:xz + org.yaml:snakeyaml com.fasterxml.jackson.core:jackson-annotations @@ -180,23 +176,45 @@ ** - + org/bouncycastle/** + + *:* + + **/module-info.class + findbugsExclude.xml + META-INF/*-LICENSE + META-INF/*-NOTICE + META-INF/*.DSA + META-INF/*.RSA + META-INF/*.SF + META-INF/DEPENDENCIES* + META-INF/io.netty.versions.properties + META-INF/LICENSE* + META-INF/license/** + META-INF/MANIFEST.MF + META-INF/maven/** + META-INF/native-image/** + META-INF/NOTICE* + META-INF/proguard/** + + + - org.asynchttpclient - org.apache.pulsar.shade.org.asynchttpclient - - - org.apache.commons - org.apache.pulsar.shade.org.apache.commons + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true - io.airlift - org.apache.pulsar.shade.io.airlift + com.fasterxml.jackson + org.apache.pulsar.shade.com.fasterxml.jackson + + com.fasterxml.jackson.annotation.* + com.google @@ -210,63 +228,67 @@ org.apache.pulsar.shade.com.spotify.futures - com.fasterxml.jackson - org.apache.pulsar.shade.com.fasterxml.jackson - - com.fasterxml.jackson.annotation.* - + com.sun.activation + org.apache.pulsar.shade.com.sun.activation - io.netty - org.apache.pulsar.shade.io.netty + com.thoughtworks.paranamer + org.apache.pulsar.shade.com.thoughtworks.paranamer - org.checkerframework - org.apache.pulsar.shade.org.checkerframework + com.typesafe + org.apache.pulsar.shade.com.typesafe - javax.annotation - org.apache.pulsar.shade.javax.annotation + com.yahoo.datasketches + org.apache.pulsar.shade.com.yahoo.datasketches - io.swagger - org.apache.pulsar.shade.io.swagger + com.yahoo.memory + org.apache.pulsar.shade.com.yahoo.memory - org.apache.pulsar.policies - org.apache.pulsar.shade.org.apache.pulsar.policies + com.yahoo.sketches + org.apache.pulsar.shade.com.yahoo.sketches - com.yahoo.datasketches - org.apache.pulsar.shade.com.yahoo.datasketches + io.airlift + org.apache.pulsar.shade.io.airlift - com.yahoo.sketches - org.apache.pulsar.shade.com.yahoo.sketches + io.netty + org.apache.pulsar.shade.io.netty - org.eclipse.jetty - org.apache.pulsar.shade.org.eclipse + io.swagger + org.apache.pulsar.shade.io.swagger - org.reactivestreams - org.apache.pulsar.shade.org.reactivestreams + javax.activation + org.apache.pulsar.shade.javax.activation - com.typesafe - org.apache.pulsar.shade.com.typesafe + javax.annotation + org.apache.pulsar.shade.javax.annotation - com.yahoo.memory - org.apache.pulsar.shade.com.yahoo.memory + javax.ws + org.apache.pulsar.shade.javax.ws - org.objenesis - org.apache.pulsar.shade.org.objenesis + META-INF/versions/(\d+)/com/fasterxml/jackson/core/ + META-INF/versions/$1/org/apache/pulsar/shade/com/fasterxml/jackson/core/ + + true - org.yaml - org.apache.pulsar.shade.org.yaml + META-INF/versions/(\d+)/org/yaml/ + META-INF/versions/$1/org/apache/pulsar/shade/org/yaml/ + true + + + net.jcip + org.apache.pulsar.shade.net.jcip org.apache.avro @@ -284,43 +306,65 @@ org.apache.avro.reflect.Union - + + org.apache.bookkeeper + org.apache.pulsar.shade.org.apache.bookkeeper + + + org.apache.commons + org.apache.pulsar.shade.org.apache.commons + + + org.apache.pulsar.policies + org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.asynchttpclient + org.apache.pulsar.shade.org.asynchttpclient + + + org.checkerframework + org.apache.pulsar.shade.org.checkerframework + + org.codehaus.jackson org.apache.pulsar.shade.org.codehaus.jackson - com.thoughtworks.paranamer - org.apache.pulsar.shade.com.thoughtworks.paranamer + org.eclipse.jetty + org.apache.pulsar.shade.org.eclipse - org.apache.commons - org.apache.pulsar.shade.org.apache.commons + org.objenesis + org.apache.pulsar.shade.org.objenesis - org.tukaani - org.apache.pulsar.shade.org.tukaani + org.reactivestreams + org.apache.pulsar.shade.org.reactivestreams - org.apache.bookkeeper - org.apache.pulsar.shade.org.apache.bookkeeper + org.tukaani + org.apache.pulsar.shade.org.tukaani - - (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ - $1org_apache_pulsar_shade_$3 - true + org.yaml + org.apache.pulsar.shade.org.yaml - - + + + + + true + + - com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 241dd173ea9a4..c823b8408c9bd 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -184,8 +184,8 @@ - javax.ws.rs - javax.ws.rs-api + jakarta.ws.rs + jakarta.ws.rs-api diff --git a/pulsar-functions/proto/pom.xml b/pulsar-functions/proto/pom.xml index db87ff7ef2031..e9e9678d1b9b7 100644 --- a/pulsar-functions/proto/pom.xml +++ b/pulsar-functions/proto/pom.xml @@ -44,10 +44,10 @@ com.google.protobuf protobuf-java-util - + - javax.annotation - javax.annotation-api + jakarta.annotation + jakarta.annotation-api diff --git a/pulsar-functions/runtime/pom.xml b/pulsar-functions/runtime/pom.xml index b04d25a5af08a..b2df8f224bc26 100644 --- a/pulsar-functions/runtime/pom.xml +++ b/pulsar-functions/runtime/pom.xml @@ -77,6 +77,10 @@ bcprov-jdk18on org.bouncycastle + + javax.annotation + javax.annotation-api + @@ -106,7 +110,7 @@ - + org.apache.maven.plugins diff --git a/pulsar-functions/secrets/pom.xml b/pulsar-functions/secrets/pom.xml index c7ab69ec612db..ac7d89a0a0c27 100644 --- a/pulsar-functions/secrets/pom.xml +++ b/pulsar-functions/secrets/pom.xml @@ -48,6 +48,10 @@ bcprov-jdk18on org.bouncycastle + + javax.annotation + javax.annotation-api + @@ -76,7 +80,7 @@ - + com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 73ed347d24fee..2cea6e0893005 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -134,8 +134,8 @@ - javax.xml.bind - jaxb-api + jakarta.xml.bind + jakarta.xml.bind-api diff --git a/tiered-storage/jcloud/pom.xml b/tiered-storage/jcloud/pom.xml index 366ba9ae38cc9..8fa504227feb1 100644 --- a/tiered-storage/jcloud/pom.xml +++ b/tiered-storage/jcloud/pom.xml @@ -104,8 +104,8 @@ - javax.xml.bind - jaxb-api + jakarta.xml.bind + jakarta.xml.bind-api javax.activation From 280997e688722be7240e777d5ca3500362c689bf Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Fri, 29 Nov 2024 22:30:16 +0800 Subject: [PATCH 174/327] [fix][broker] support missing tenant level fine-granted permissions (#23660) --- .../pulsar/broker/admin/impl/TenantsBase.java | 48 +++++- .../TenantEndpointsAuthorizationTest.java | 160 ++++++++++++++++++ .../common/policies/data/TenantOperation.java | 6 + 3 files changed, 209 insertions(+), 5 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TenantEndpointsAuthorizationTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java index 0d1f79a09dc14..ff32e41977aaa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java @@ -48,6 +48,7 @@ import org.apache.pulsar.common.naming.NamedEntity; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TenantOperation; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,7 +63,7 @@ public class TenantsBase extends PulsarWebResource { @ApiResponse(code = 404, message = "Tenant doesn't exist")}) public void getTenants(@Suspended final AsyncResponse asyncResponse) { final String clientAppId = clientAppId(); - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(null, TenantOperation.LIST_TENANTS) .thenCompose(__ -> tenantResources().listTenantsAsync()) .thenAccept(tenants -> { // deep copy the tenants to avoid concurrent sort exception @@ -84,7 +85,7 @@ public void getTenants(@Suspended final AsyncResponse asyncResponse) { public void getTenantAdmin(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "The tenant name") @PathParam("tenant") String tenant) { final String clientAppId = clientAppId(); - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(tenant, TenantOperation.GET_TENANT) .thenCompose(__ -> tenantResources().getTenantAsync(tenant)) .thenApply(tenantInfo -> { if (!tenantInfo.isPresent()) { @@ -121,7 +122,7 @@ public void createTenant(@Suspended final AsyncResponse asyncResponse, asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, "Tenant name is not valid")); return; } - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(tenant, TenantOperation.CREATE_TENANT) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> validateClustersAsync(tenantInfo)) .thenCompose(__ -> validateAdminRoleAsync(tenantInfo)) @@ -169,7 +170,7 @@ public void updateTenant(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "The tenant name") @PathParam("tenant") String tenant, @ApiParam(value = "TenantInfo") TenantInfoImpl newTenantAdmin) { final String clientAppId = clientAppId(); - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(tenant, TenantOperation.UPDATE_TENANT) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> validateClustersAsync(newTenantAdmin)) .thenCompose(__ -> validateAdminRoleAsync(newTenantAdmin)) @@ -206,7 +207,7 @@ public void deleteTenant(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") @ApiParam(value = "The tenant name") String tenant, @QueryParam("force") @DefaultValue("false") boolean force) { final String clientAppId = clientAppId(); - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(tenant, TenantOperation.DELETE_TENANT) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> internalDeleteTenant(tenant, force)) .thenAccept(__ -> { @@ -304,4 +305,41 @@ private CompletableFuture validateAdminRoleAsync(TenantInfoImpl info) { } return CompletableFuture.completedFuture(null); } + + private CompletableFuture validateBothSuperUserAndTenantOperation(String tenant, + TenantOperation operation) { + final var superUserValidationFuture = validateSuperUserAccessAsync(); + final var tenantOperationValidationFuture = validateTenantOperationAsync(tenant, operation); + return CompletableFuture.allOf(superUserValidationFuture, tenantOperationValidationFuture) + .handle((__, err) -> { + if (!superUserValidationFuture.isCompletedExceptionally() + || !tenantOperationValidationFuture.isCompletedExceptionally()) { + return true; + } + if (log.isDebugEnabled()) { + Throwable superUserValidationException = null; + try { + superUserValidationFuture.join(); + } catch (Throwable ex) { + superUserValidationException = FutureUtil.unwrapCompletionException(ex); + } + Throwable brokerOperationValidationException = null; + try { + tenantOperationValidationFuture.join(); + } catch (Throwable ex) { + brokerOperationValidationException = FutureUtil.unwrapCompletionException(ex); + } + log.debug("validateBothTenantOperationAndSuperUser failed." + + " originalPrincipal={} clientAppId={} operation={} " + + "superuserValidationError={} tenantOperationValidationError={}", + originalPrincipal(), clientAppId(), operation.toString(), + superUserValidationException, brokerOperationValidationException); + } + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBothTenantOperationAndSuperUser for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] ", + originalPrincipal(), clientAppId(), operation.toString())); + }); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TenantEndpointsAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TenantEndpointsAuthorizationTest.java new file mode 100644 index 0000000000000..2cf3ea374c33c --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TenantEndpointsAuthorizationTest.java @@ -0,0 +1,160 @@ +/* + * 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.admin; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isNull; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TenantOperation; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import java.util.Set; +import java.util.UUID; + +@Test(groups = "broker-admin") +public class TenantEndpointsAuthorizationTest extends MockedPulsarStandalone { + + private AuthorizationService orignalAuthorizationService; + private AuthorizationService spyAuthorizationService; + + private PulsarAdmin superUserAdmin; + private PulsarAdmin nobodyAdmin; + + @SneakyThrows + @BeforeClass(alwaysRun = true) + public void setup() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + start(); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + this.nobodyAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(NOBODY_TOKEN)) + .build(); + } + + @BeforeMethod(alwaysRun = true) + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + spyAuthorizationService = spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + spyAuthorizationService, true); + } + + @AfterMethod(alwaysRun = true) + public void after() throws IllegalAccessException { + if (orignalAuthorizationService != null) { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", orignalAuthorizationService, true); + } + } + + @SneakyThrows + @AfterClass(alwaysRun = true) + public void cleanup() { + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + spyAuthorizationService = null; + orignalAuthorizationService = null; + super.close(); + } + + @Test + public void testListTenants() throws PulsarAdminException { + superUserAdmin.tenants().getTenants(); + // test allow broker operation + verify(spyAuthorizationService) + .allowTenantOperationAsync(isNull(), Mockito.eq(TenantOperation.LIST_TENANTS), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.tenants().getTenants()); + } + + + @Test + public void testGetTenant() throws PulsarAdminException { + String tenantName = "public"; + superUserAdmin.tenants().getTenantInfo(tenantName); + // test allow broker operation + verify(spyAuthorizationService) + .allowTenantOperationAsync(eq(tenantName), Mockito.eq(TenantOperation.GET_TENANT), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.tenants().getTenantInfo(tenantName)); + } + + @Test + public void testUpdateTenant() throws PulsarAdminException { + String tenantName = "public"; + superUserAdmin.tenants().updateTenant(tenantName, TenantInfo.builder() + .allowedClusters(Set.of(getPulsarService().getConfiguration().getClusterName())) + .adminRoles(Set.of("example")).build()); + // test allow broker operation + verify(spyAuthorizationService) + .allowTenantOperationAsync(eq(tenantName), Mockito.eq(TenantOperation.UPDATE_TENANT), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.tenants() + .updateTenant(tenantName, TenantInfo.builder().adminRoles(Set.of("example")).build())); + } + + @Test + public void testDeleteTenant() throws PulsarAdminException { + String tenantName = UUID.randomUUID().toString(); + superUserAdmin.tenants().createTenant(tenantName, TenantInfo.builder() + .allowedClusters(Set.of(getPulsarService().getConfiguration().getClusterName())) + .adminRoles(Set.of("example")).build()); + + Mockito.clearInvocations(spyAuthorizationService); + superUserAdmin.tenants().deleteTenant(tenantName); + // test allow broker operation + verify(spyAuthorizationService) + .allowTenantOperationAsync(eq(tenantName), Mockito.eq(TenantOperation.DELETE_TENANT), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.tenants().deleteTenant(tenantName)); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantOperation.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantOperation.java index 1c52f69006403..e0518e510f9dc 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantOperation.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantOperation.java @@ -25,4 +25,10 @@ public enum TenantOperation { CREATE_NAMESPACE, DELETE_NAMESPACE, LIST_NAMESPACES, + + LIST_TENANTS, + GET_TENANT, + CREATE_TENANT, + UPDATE_TENANT, + DELETE_TENANT, } From bf1f67742243f1b40c258c64e1fd6a960611a780 Mon Sep 17 00:00:00 2001 From: hanmz Date: Fri, 29 Nov 2024 22:56:03 +0800 Subject: [PATCH 175/327] [fix][client] Fix race-condition causing doReconsumeLater to hang when creating retryLetterProducer has failed (#23560) --- .../pulsar/client/api/RetryTopicTest.java | 68 +++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 2 + 2 files changed, 70 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 9cb82fde04118..cd598585c8e87 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 @@ -30,6 +30,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.Data; @@ -45,6 +46,7 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import org.testng.collections.Lists; @Test(groups = "broker-api") public class RetryTopicTest extends ProducerConsumerBase { @@ -713,4 +715,70 @@ public void testRetryProducerWillCloseByConsumer() throws Exception { admin.topics().delete(topicDLQ, false); } + + @Test(timeOut = 30000L) + public void testRetryTopicExceptionWithConcurrent() throws Exception { + final String topic = "persistent://my-property/my-ns/retry-topic"; + final int maxRedeliveryCount = 2; + final int sendMessages = 10; + // subscribe before publish + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) + .topic(topic) + .subscriptionName("my-subscription") + .subscriptionType(SubscriptionType.Shared) + .enableRetry(true) + .receiverQueueSize(100) + .deadLetterPolicy(DeadLetterPolicy.builder() + .maxRedeliverCount(maxRedeliveryCount) + .retryLetterTopic("persistent://my-property/my-ns/my-subscription-custom-Retry") + .build()) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .create(); + for (int i = 0; i < sendMessages; i++) { + producer.newMessage().key("1").value(String.format("Hello Pulsar [%d]", i).getBytes()).send(); + } + producer.close(); + + // mock a retry producer exception when reconsumelater is called + MultiTopicsConsumerImpl multiTopicsConsumer = (MultiTopicsConsumerImpl) consumer; + List> consumers = multiTopicsConsumer.getConsumers(); + for (ConsumerImpl c : consumers) { + Set deadLetterPolicyField = + ReflectionUtils.getAllFields(c.getClass(), ReflectionUtils.withName("deadLetterPolicy")); + + if (deadLetterPolicyField.size() != 0) { + Field field = deadLetterPolicyField.iterator().next(); + field.setAccessible(true); + DeadLetterPolicy deadLetterPolicy = (DeadLetterPolicy) field.get(c); + deadLetterPolicy.setRetryLetterTopic("#persistent://invalid-topic#"); + } + } + + List> messages = Lists.newArrayList(); + for (int i = 0; i < sendMessages; i++) { + messages.add(consumer.receive()); + } + + // mock call the reconsumeLater method concurrently + CountDownLatch latch = new CountDownLatch(messages.size()); + for (Message message : messages) { + new Thread(() -> { + try { + consumer.reconsumeLater(message, 1, TimeUnit.SECONDS); + } catch (Exception ignore) { + + } finally { + latch.countDown(); + } + }).start(); + } + + latch.await(); + consumer.close(); + } + } 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 390a70095182f..d2753856264fc 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 @@ -752,6 +752,8 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a } catch (Exception e) { result.completeExceptionally(e); } + } else { + result.completeExceptionally(new PulsarClientException("Retry letter producer is null.")); } MessageId finalMessageId = messageId; result.exceptionally(ex -> { From 7fc88d650202bca5d0462b41a89c5189f90a3859 Mon Sep 17 00:00:00 2001 From: Yike Xiao Date: Fri, 29 Nov 2024 23:03:55 +0800 Subject: [PATCH 176/327] [improve][client] Enhance error handling for non-exist subscription in consumer creation (#23254) --- .../apache/pulsar/client/api/MultiTopicsConsumerTest.java | 6 ++++-- .../org/apache/pulsar/client/api/PulsarClientException.java | 2 ++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java index 7a12acd47edf9..ea8eb6e8cc081 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java @@ -413,8 +413,9 @@ public void testSubscriptionNotFound() throws PulsarAdminException, PulsarClient .isAckReceiptEnabled(true) .subscribe(); assertTrue(singleTopicConsumer instanceof ConsumerImpl); + } catch (PulsarClientException.SubscriptionNotFoundException ignore) { } catch (Throwable t) { - assertTrue(t.getCause().getCause() instanceof PulsarClientException.SubscriptionNotFoundException); + fail("Should throw PulsarClientException.SubscriptionNotFoundException instead"); } try { @@ -424,8 +425,9 @@ public void testSubscriptionNotFound() throws PulsarAdminException, PulsarClient .isAckReceiptEnabled(true) .subscribe(); assertTrue(multiTopicsConsumer instanceof MultiTopicsConsumerImpl); + } catch (PulsarClientException.SubscriptionNotFoundException ignore) { } catch (Throwable t) { - assertTrue(t.getCause().getCause() instanceof PulsarClientException.SubscriptionNotFoundException); + fail("Should throw PulsarClientException.SubscriptionNotFoundException instead"); } pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java index 9eb6c612a52a2..b2c9b2b697b42 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java @@ -1113,6 +1113,8 @@ public static PulsarClientException unwrap(Throwable t) { newException = new TransactionConflictException(msg); } else if (cause instanceof TopicDoesNotExistException) { newException = new TopicDoesNotExistException(msg); + } else if (cause instanceof SubscriptionNotFoundException) { + newException = new SubscriptionNotFoundException(msg); } else if (cause instanceof ProducerFencedException) { newException = new ProducerFencedException(msg); } else if (cause instanceof MemoryBufferIsFullError) { From 46037229947c8031207eaed70dd937e6990de544 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Sat, 30 Nov 2024 01:17:16 +0800 Subject: [PATCH 177/327] [feat][broker] Implement allowBrokerOperationAsync in PulsarAuthorizationProvider to avoid exception thrown (#23663) --- .../broker/authorization/PulsarAuthorizationProvider.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java index 0af63724cc812..50783c4d1338b 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java @@ -40,6 +40,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.AuthPolicies; +import org.apache.pulsar.common.policies.data.BrokerOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -690,6 +691,13 @@ public CompletableFuture allowTopicOperationAsync(TopicName topicName, }); } + @Override + public CompletableFuture allowBrokerOperationAsync(String clusterName, String brokerId, + BrokerOperation brokerOperation, String role, + AuthenticationDataSource authData) { + return isSuperUser(role, authData, conf); + } + @Override public CompletableFuture allowTopicPolicyOperationAsync(TopicName topicName, String role, PolicyName policyName, From f27905560207eb2ade32f8086b4585dffb918b80 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 2 Dec 2024 17:21:32 +0200 Subject: [PATCH 178/327] [improve] Install coreutils in docker image to improve compatibility (#23667) --- docker/pulsar/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index f8c22dc14a821..c8fc3a4d13c8f 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -94,7 +94,8 @@ RUN apk add --no-cache \ procps \ curl \ bind-tools \ - openssl + openssl \ + coreutils # Upgrade all packages to get latest versions with security fixes RUN apk upgrade --no-cache From 59d4588c1924df6e6a5e9eb9c525f6db0beeddf4 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Wed, 4 Dec 2024 10:01:44 +0800 Subject: [PATCH 179/327] [fix] [broker] Add consumer name for subscription stats (#23671) --- .../src/main/java/org/apache/pulsar/broker/service/Consumer.java | 1 + 1 file changed, 1 insertion(+) 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 bcd29d86490cf..0baba13cc0b07 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 @@ -977,6 +977,7 @@ public ConsumerStatsImpl getStats() { stats.unackedMessages = unackedMessages; stats.blockedConsumerOnUnackedMsgs = blockedConsumerOnUnackedMsgs; stats.avgMessagesPerEntry = getAvgMessagesPerEntry(); + stats.consumerName = consumerName; if (readPositionWhenJoining != null) { stats.readPositionWhenJoining = readPositionWhenJoining.toString(); } From 1c1a5cc655511c22b3399005d8ff9102b3553627 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Wed, 4 Dec 2024 10:16:21 +0800 Subject: [PATCH 180/327] [fix][broker] support missing cluster level fine-granted permissions (#23675) --- .../authorization/AuthorizationProvider.java | 20 + .../authorization/AuthorizationService.java | 45 ++ .../PulsarAuthorizationProvider.java | 15 + .../broker/admin/impl/ClustersBase.java | 160 ++++++- .../ClusterEndpointsAuthorizationTest.java | 428 ++++++++++++++++++ .../policies/data/ClusterOperation.java | 34 ++ .../common/policies/data/PolicyName.java | 6 +- 7 files changed, 689 insertions(+), 19 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ClusterEndpointsAuthorizationTest.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterOperation.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java index 48386265940a3..cb61292f8e38e 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java @@ -33,6 +33,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.BrokerOperation; +import org.apache.pulsar.common.policies.data.ClusterOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -393,4 +394,23 @@ default CompletableFuture allowBrokerOperationAsync(String clusterName, return FutureUtil.failedFuture( new UnsupportedOperationException("allowBrokerOperationAsync is not supported yet.")); } + + + default CompletableFuture allowClusterOperationAsync(String clusterName, + ClusterOperation clusterOperation, + String role, + AuthenticationDataSource authData) { + return FutureUtil.failedFuture( + new UnsupportedOperationException("allowClusterOperationAsync is not supported yet.")); + } + + default CompletableFuture allowClusterPolicyOperationAsync(String clusterName, + String role, + PolicyName policy, + PolicyOperation operation, + AuthenticationDataSource authData) { + return FutureUtil.failedFuture( + new IllegalStateException("ClusterPolicyOperation [" + policy.name() + "/" + operation.name() + "] " + + "is not supported by the Authorization provider you are using.")); + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java index 1348a405b0dfa..40573d99d6012 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java @@ -39,6 +39,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.BrokerOperation; +import org.apache.pulsar.common.policies.data.ClusterOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -567,6 +568,50 @@ public CompletableFuture allowBrokerOperationAsync(String clusterName, } } + public CompletableFuture allowClusterOperationAsync(String clusterName, + ClusterOperation clusterOperation, + String originalRole, + String role, + AuthenticationDataSource authData) { + if (!isValidOriginalPrincipal(role, originalRole, authData)) { + return CompletableFuture.completedFuture(false); + } + + if (isProxyRole(role)) { + final var isRoleAuthorizedFuture = provider.allowClusterOperationAsync(clusterName, + clusterOperation, role, authData); + final var isOriginalAuthorizedFuture = provider.allowClusterOperationAsync(clusterName, + clusterOperation, originalRole, authData); + return isRoleAuthorizedFuture.thenCombine(isOriginalAuthorizedFuture, + (isRoleAuthorized, isOriginalAuthorized) -> isRoleAuthorized && isOriginalAuthorized); + } else { + return provider.allowClusterOperationAsync(clusterName, clusterOperation, role, authData); + } + } + + public CompletableFuture allowClusterPolicyOperationAsync(String clusterName, + PolicyName policy, + PolicyOperation operation, + String originalRole, + String role, + AuthenticationDataSource authData) { + if (!isValidOriginalPrincipal(role, originalRole, authData)) { + return CompletableFuture.completedFuture(false); + } + + if (isProxyRole(role)) { + final var isRoleAuthorizedFuture = provider.allowClusterPolicyOperationAsync(clusterName, role, + policy, operation, authData); + final var isOriginalAuthorizedFuture = provider.allowClusterPolicyOperationAsync(clusterName, originalRole, + policy, operation, authData); + return isRoleAuthorizedFuture.thenCombine(isOriginalAuthorizedFuture, + (isRoleAuthorized, isOriginalAuthorized) -> isRoleAuthorized && isOriginalAuthorized); + } else { + return provider.allowClusterPolicyOperationAsync(clusterName, role, policy, operation, authData); + } + } + + /** * @deprecated - will be removed after 2.12. Use async variant. */ diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java index 50783c4d1338b..976e7b7ee12e7 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java @@ -41,6 +41,7 @@ import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.AuthPolicies; import org.apache.pulsar.common.policies.data.BrokerOperation; +import org.apache.pulsar.common.policies.data.ClusterOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -860,4 +861,18 @@ public CompletableFuture>> getPermissionsAsync(Names }); }); } + + @Override + public CompletableFuture allowClusterOperationAsync(String clusterName, ClusterOperation clusterOperation, + String role, AuthenticationDataSource authData) { + return isSuperUser(role, authData, conf); + } + + @Override + public CompletableFuture allowClusterPolicyOperationAsync(String clusterName, String role, + PolicyName policy, + PolicyOperation operation, + AuthenticationDataSource authData) { + return isSuperUser(role, authData, conf); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java index b261033ca52c9..d24a3255b5556 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java @@ -63,11 +63,14 @@ import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.ClusterOperation; import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl; import org.apache.pulsar.common.policies.data.ClusterPoliciesImpl; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.NamespaceIsolationPolicyUnloadScope; +import org.apache.pulsar.common.policies.data.PolicyName; +import org.apache.pulsar.common.policies.data.PolicyOperation; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicyImpl; import org.apache.pulsar.common.util.FutureUtil; @@ -117,7 +120,7 @@ public void getClusters(@Suspended AsyncResponse asyncResponse) { public void getCluster(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.GET_CLUSTER) .thenCompose(__ -> clusterResources().getClusterAsync(cluster)) .thenAccept(clusterData -> { asyncResponse.resume(clusterData @@ -162,7 +165,7 @@ public void createCluster( ) ) ) ClusterDataImpl clusterData) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.CREATE_CLUSTER) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> { NamedEntity.checkName(cluster); @@ -227,7 +230,7 @@ public void updateCluster( ) ) ) ClusterDataImpl clusterData) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.UPDATE_CLUSTER) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> { try { @@ -271,7 +274,7 @@ public void getClusterMigration( required = true ) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.CLUSTER_MIGRATION, PolicyOperation.READ) .thenCompose(__ -> clusterResources().getClusterPoliciesResources().getClusterPoliciesAsync(cluster)) .thenAccept(policies -> { asyncResponse.resume( @@ -326,7 +329,7 @@ public void updateClusterMigration( asyncResponse.resume(new RestException(Status.BAD_REQUEST, "Cluster url must not be empty")); return; } - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.CLUSTER_MIGRATION, PolicyOperation.WRITE) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> clusterResources().getClusterPoliciesResources().setPoliciesWithCreateAsync(cluster, old -> { @@ -376,7 +379,7 @@ public void setPeerClusterNames(@Suspended AsyncResponse asyncResponse, "cluster-b" ]"""))) LinkedHashSet peerClusterNames) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.UPDATE_PEER_CLUSTER) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> innerSetPeerClusterNamesAsync(cluster, peerClusterNames)) .thenAccept(__ -> { @@ -437,7 +440,7 @@ private CompletableFuture innerSetPeerClusterNamesAsync(String cluster, public void getPeerCluster(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.GET_PEER_CLUSTER) .thenCompose(__ -> clusterResources().getClusterAsync(cluster)) .thenAccept(clusterOpt -> { ClusterData clusterData = @@ -466,7 +469,7 @@ public void getPeerCluster(@Suspended AsyncResponse asyncResponse, public void deleteCluster(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.DELETE_CLUSTER) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> internalDeleteClusterAsync(cluster)) .thenAccept(__ -> { @@ -525,7 +528,7 @@ public void getNamespaceIsolationPolicies( @Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.READ) .thenCompose(__ -> validateClusterExistAsync(cluster, Status.NOT_FOUND)) .thenCompose(__ -> internalGetNamespaceIsolationPolicies(cluster)) .thenAccept(asyncResponse::resume) @@ -583,7 +586,7 @@ public void getNamespaceIsolationPolicy( @ApiParam(value = "The name of the namespace isolation policy", required = true) @PathParam("policyName") String policyName ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.READ) .thenCompose(__ -> validateClusterExistAsync(cluster, Status.PRECONDITION_FAILED)) .thenCompose(__ -> internalGetNamespaceIsolationPolicies(cluster)) .thenAccept(policies -> { @@ -619,7 +622,7 @@ public void getBrokersWithNamespaceIsolationPolicy( @Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.READ) .thenCompose(__ -> validateClusterExistAsync(cluster, Status.PRECONDITION_FAILED)) .thenCompose(__ -> pulsar().getLoadManager().get().getAvailableBrokersAsync()) .thenCompose(availableBrokers -> internalGetNamespaceIsolationPolicies(cluster) @@ -676,7 +679,7 @@ public void getBrokerWithNamespaceIsolationPolicy( @ApiParam(value = "The broker name (:)", required = true, example = "broker1:8080") @PathParam("broker") String broker) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.READ) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> internalGetNamespaceIsolationPolicies(cluster)) .thenApply(policies -> internalGetBrokerNsIsolationData(broker, policies)) @@ -711,7 +714,7 @@ public void setNamespaceIsolationPolicy( @ApiParam(value = "The namespace isolation policy data", required = true) NamespaceIsolationDataImpl policyData ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.WRITE) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> { @@ -874,7 +877,7 @@ public void deleteNamespaceIsolationPolicy( @ApiParam(value = "The namespace isolation policy name", required = true) @PathParam("policyName") String policyName ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.WRITE) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> namespaceIsolationPolicies().getIsolationDataPoliciesAsync(cluster)) @@ -924,7 +927,7 @@ public void setFailureDomain( @PathParam("domainName") String domainName, @ApiParam(value = "The configuration data of a failure domain", required = true) FailureDomainImpl domain ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.UPDATE_FAILURE_DOMAIN) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> validateBrokerExistsInOtherDomain(cluster, domainName, domain)) .thenCompose(__ -> clusterResources().getFailureDomainResources() @@ -967,7 +970,7 @@ public void getFailureDomains( @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.GET_FAILURE_DOMAIN) .thenCompose(__ -> clusterResources().getFailureDomainResources() .listFailureDomainsAsync(cluster) .thenCompose(domainNames -> { @@ -1023,7 +1026,7 @@ public void getDomain( @ApiParam(value = "The failure domain name", required = true) @PathParam("domainName") String domainName ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.GET_FAILURE_DOMAIN) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> clusterResources().getFailureDomainResources() .getFailureDomainAsync(cluster, domainName)) @@ -1058,7 +1061,7 @@ public void deleteFailureDomain( @ApiParam(value = "The failure domain name", required = true) @PathParam("domainName") String domainName ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.DELETE_FAILURE_DOMAIN) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> clusterResources() .getFailureDomainResources().deleteFailureDomainAsync(cluster, domainName)) @@ -1124,5 +1127,126 @@ private CompletableFuture validateBrokerExistsInOtherDomain(final String c }); } + + + private CompletableFuture validateBothSuperuserAndClusterOperation(String clusterName, + ClusterOperation operation) { + final var superUserAccessValidation = validateSuperUserAccessAsync(); + final var clusterOperationValidation = validateClusterOperationAsync(clusterName, operation); + return FutureUtil.waitForAll(List.of(superUserAccessValidation, clusterOperationValidation)) + .handle((result, err) -> { + if (!superUserAccessValidation.isCompletedExceptionally() + || !clusterOperationValidation.isCompletedExceptionally()) { + return null; + } + if (log.isDebugEnabled()) { + Throwable superUserValidationException = null; + try { + superUserAccessValidation.join(); + } catch (Throwable ex) { + superUserValidationException = FutureUtil.unwrapCompletionException(ex); + } + Throwable clusterOperationValidationException = null; + try { + clusterOperationValidation.join(); + } catch (Throwable ex) { + clusterOperationValidationException = FutureUtil.unwrapCompletionException(ex); + } + log.debug("validateBothSuperuserAndClusterOperation failed." + + " originalPrincipal={} clientAppId={} operation={} cluster={} " + + "superuserValidationError={} clusterOperationValidationError={}", + originalPrincipal(), clientAppId(), operation.toString(), clusterName, + superUserValidationException, clusterOperationValidationException); + } + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBothSuperuserAndClusterOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on cluster [%s]", + originalPrincipal(), clientAppId(), operation.toString(), clusterName)); + }); + } + + + private CompletableFuture validateBothSuperuserAndClusterPolicyOperation(String clusterName, PolicyName name, + PolicyOperation operation) { + final var superUserAccessValidation = validateSuperUserAccessAsync(); + final var clusterOperationValidation = validateClusterPolicyOperationAsync(clusterName, name, operation); + return FutureUtil.waitForAll(List.of(superUserAccessValidation, clusterOperationValidation)) + .handle((result, err) -> { + if (!superUserAccessValidation.isCompletedExceptionally() + || !clusterOperationValidation.isCompletedExceptionally()) { + return null; + } + if (log.isDebugEnabled()) { + Throwable superUserValidationException = null; + try { + superUserAccessValidation.join(); + } catch (Throwable ex) { + superUserValidationException = FutureUtil.unwrapCompletionException(ex); + } + Throwable clusterOperationValidationException = null; + try { + clusterOperationValidation.join(); + } catch (Throwable ex) { + clusterOperationValidationException = FutureUtil.unwrapCompletionException(ex); + } + log.debug("validateBothSuperuserAndClusterPolicyOperation failed." + + " originalPrincipal={} clientAppId={} operation={} cluster={} " + + "superuserValidationError={} clusterOperationValidationError={}", + originalPrincipal(), clientAppId(), operation.toString(), clusterName, + superUserValidationException, clusterOperationValidationException); + } + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBothSuperuserAndClusterPolicyOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on cluster [%s]", + originalPrincipal(), clientAppId(), operation.toString(), clusterName)); + }); + } + + + + + private CompletableFuture validateClusterOperationAsync(String cluster, ClusterOperation operation) { + final var pulsar = pulsar(); + if (pulsar.getBrokerService().isAuthenticationEnabled() + && pulsar.getBrokerService().isAuthorizationEnabled()) { + return pulsar.getBrokerService().getAuthorizationService() + .allowClusterOperationAsync(cluster, operation, originalPrincipal(), + clientAppId(), clientAuthData()) + .thenAccept(isAuthorized -> { + if (!isAuthorized) { + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateClusterOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on cluster [%s]", + originalPrincipal(), clientAppId(), operation.toString(), cluster)); + } + }); + } + return CompletableFuture.completedFuture(null); + } + + private CompletableFuture validateClusterPolicyOperationAsync(String cluster, PolicyName policyName, + PolicyOperation operation) { + final var pulsar = pulsar(); + if (pulsar.getBrokerService().isAuthenticationEnabled() + && pulsar.getBrokerService().isAuthorizationEnabled()) { + return pulsar.getBrokerService().getAuthorizationService() + .allowClusterPolicyOperationAsync(cluster, policyName, operation, originalPrincipal(), + clientAppId(), clientAuthData()) + .thenAccept(isAuthorized -> { + if (!isAuthorized) { + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateClusterPolicyOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on cluster [%s]", + originalPrincipal(), clientAppId(), operation.toString(), cluster)); + } + }); + } + return CompletableFuture.completedFuture(null); + } + private static final Logger log = LoggerFactory.getLogger(ClustersBase.class); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ClusterEndpointsAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ClusterEndpointsAuthorizationTest.java new file mode 100644 index 0000000000000..ccf5ccb9481ec --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ClusterEndpointsAuthorizationTest.java @@ -0,0 +1,428 @@ +/* + * 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.admin; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.ClusterOperation; +import org.apache.pulsar.common.policies.data.ClusterPolicies; +import org.apache.pulsar.common.policies.data.FailureDomain; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; +import org.apache.pulsar.common.policies.data.PolicyName; +import org.apache.pulsar.common.policies.data.PolicyOperation; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import java.util.LinkedHashSet; +import java.util.UUID; + + +@Test(groups = "broker-admin") +public class ClusterEndpointsAuthorizationTest extends MockedPulsarStandalone { + + private AuthorizationService orignalAuthorizationService; + private AuthorizationService spyAuthorizationService; + + private PulsarAdmin superUserAdmin; + private PulsarAdmin nobodyAdmin; + + @SneakyThrows + @BeforeClass(alwaysRun = true) + public void setup() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + start(); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + this.nobodyAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(NOBODY_TOKEN)) + .build(); + } + + @BeforeMethod(alwaysRun = true) + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + spyAuthorizationService = spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + spyAuthorizationService, true); + } + + @AfterMethod(alwaysRun = true) + public void after() throws IllegalAccessException { + if (orignalAuthorizationService != null) { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", orignalAuthorizationService, true); + } + } + + @SneakyThrows + @AfterClass(alwaysRun = true) + public void cleanup() { + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + spyAuthorizationService = null; + orignalAuthorizationService = null; + super.close(); + } + + + @Test + public void testGetCluster() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getCluster(clusterName); + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.GET_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getCluster(clusterName)); + } + + + @Test + public void testCreateCluster() throws PulsarAdminException { + final String clusterName = UUID.randomUUID().toString(); + superUserAdmin.clusters().createCluster(clusterName, ClusterData.builder().build()); + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.CREATE_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().createCluster(clusterName, ClusterData.builder().build())); + } + + @Test + public void testUpdateCluster() { + final String clusterName = UUID.randomUUID().toString(); + try { + superUserAdmin.clusters().updateCluster(clusterName, ClusterData.builder().serviceUrl("aaa").build()); + } catch (Throwable ignore) { + + } + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.UPDATE_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().updateCluster(clusterName, ClusterData.builder().build())); + } + + + @Test + public void testGetClusterMigration() { + final String clusterName = UUID.randomUUID().toString(); + try { + superUserAdmin.clusters().getClusterMigration(clusterName); + } catch (Throwable ignore) { + + } + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), eq(PolicyName.CLUSTER_MIGRATION), + eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getClusterMigration(clusterName)); + } + + + @Test + public void testUpdateClusterMigration() throws PulsarAdminException { + final String clusterName = UUID.randomUUID().toString(); + superUserAdmin.clusters().createCluster(clusterName, ClusterData.builder().build()); + Mockito.clearInvocations(spyAuthorizationService); + + superUserAdmin.clusters().updateClusterMigration(clusterName, false, new ClusterPolicies.ClusterUrl()); + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), eq(PolicyName.CLUSTER_MIGRATION), + eq(PolicyOperation.WRITE), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters() + .updateClusterMigration(clusterName, false, new ClusterPolicies.ClusterUrl())); + } + + @Test + public void testSetPeerClusterNames() throws PulsarAdminException { + final LinkedHashSet linkedHashSet = new LinkedHashSet<>(); + linkedHashSet.add("a"); + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + try { + superUserAdmin.clusters().updatePeerClusterNames(clusterName, linkedHashSet); + } catch (Throwable ignore) { + + } + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), + eq(ClusterOperation.UPDATE_PEER_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().updatePeerClusterNames(clusterName, linkedHashSet)); + } + + @Test + public void testGetPeerCluster() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getPeerClusterNames(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), + eq(ClusterOperation.GET_PEER_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getPeerClusterNames(clusterName)); + } + + @Test + public void testDeleteCluster() throws PulsarAdminException { + final String clusterName = UUID.randomUUID().toString(); + superUserAdmin.clusters().createCluster(clusterName, ClusterData.builder().build()); + Mockito.clearInvocations(spyAuthorizationService); + + superUserAdmin.clusters().deleteCluster(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), + eq(ClusterOperation.DELETE_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().deleteCluster(clusterName)); + } + + + @Test + public void testGetNamespaceIsolationPolicies() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getNamespaceIsolationPolicies(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getNamespaceIsolationPolicies(clusterName)); + } + + + @Test + public void testGetNamespaceIsolationPolicy() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getNamespaceIsolationPolicy(clusterName, ""); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getNamespaceIsolationPolicy(clusterName, "")); + } + + + @Test + public void testGetBrokersWithNamespaceIsolationPolicy() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getBrokersWithNamespaceIsolationPolicy(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getBrokersWithNamespaceIsolationPolicy(clusterName)); + } + + + @Test + public void testGetBrokerWithNamespaceIsolationPolicy() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getBrokerWithNamespaceIsolationPolicy(clusterName, getPulsarService().getBrokerId()); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getBrokerWithNamespaceIsolationPolicy(clusterName, "")); + } + + + @Test + public void testSetNamespaceIsolationPolicy() { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + + try { + superUserAdmin.clusters().updateNamespaceIsolationPolicy(clusterName, "test", + NamespaceIsolationData.builder().build()); + } catch (Throwable ignore) { + + } + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.WRITE), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().updateNamespaceIsolationPolicy(clusterName, "test", + NamespaceIsolationData.builder().build())); + } + + @Test + public void testDeleteNamespaceIsolationPolicy() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().deleteNamespaceIsolationPolicy(clusterName, "test"); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.WRITE), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().deleteNamespaceIsolationPolicy(clusterName, "test")); + } + + + @Test + public void testSetFailureDomain() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().updateFailureDomain(clusterName, "test", FailureDomain.builder().build()); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.UPDATE_FAILURE_DOMAIN), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().updateFailureDomain(clusterName, + "test", FailureDomain.builder().build())); + } + + @Test + public void testGetFailureDomains() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getFailureDomains(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.GET_FAILURE_DOMAIN), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getFailureDomains(clusterName)); + } + + + @Test + public void testGetDomain() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + try { + superUserAdmin.clusters().getFailureDomain(clusterName, "test"); + } catch (Throwable ignore) { + + } + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.GET_FAILURE_DOMAIN), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getFailureDomain(clusterName, "test")); + } + + @Test + public void testDeleteFailureDomain() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + try { + superUserAdmin.clusters().deleteFailureDomain(clusterName, "test"); + } catch (Throwable ignore) { + + } + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.DELETE_FAILURE_DOMAIN), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().deleteFailureDomain(clusterName, "test")); + } + + +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterOperation.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterOperation.java new file mode 100644 index 0000000000000..bbdc64f729ef4 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterOperation.java @@ -0,0 +1,34 @@ +/* + * 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.common.policies.data; + +public enum ClusterOperation { + LIST_CLUSTERS, + GET_CLUSTER, + CREATE_CLUSTER, + UPDATE_CLUSTER, + DELETE_CLUSTER, + + // detailed update + GET_PEER_CLUSTER, + UPDATE_PEER_CLUSTER, + GET_FAILURE_DOMAIN, + UPDATE_FAILURE_DOMAIN, + DELETE_FAILURE_DOMAIN +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java index 86ab545215e99..d77f92eb03292 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java @@ -53,5 +53,9 @@ public enum PolicyName { ENTRY_FILTERS, SHADOW_TOPIC, DISPATCHER_PAUSE_ON_ACK_STATE_PERSISTENT, - ALLOW_CLUSTERS + ALLOW_CLUSTERS, + + // cluster policies + CLUSTER_MIGRATION, + NAMESPACE_ISOLATION, } From 7f7e12bf6e1a74119b87be2c85a509a935b20e57 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Wed, 4 Dec 2024 18:13:50 +0800 Subject: [PATCH 181/327] [fix][broker] Invoke custom BrokerInterceptor's `onFilter` method if it's defined (#23676) --- .../BrokerInterceptorWithClassLoader.java | 13 +++++++++++++ .../org/apache/pulsar/broker/web/WebService.java | 14 +++++++++++++- .../plugins/LoggingBrokerInterceptor.java | 6 +++++- .../plugins/TestBrokerInterceptors.java | 1 + 4 files changed, 32 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java index 3997e214f4316..849f7aa39f0ef 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java @@ -22,6 +22,7 @@ import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.Map; +import javax.servlet.FilterChain; import javax.servlet.ServletException; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; @@ -272,6 +273,18 @@ public void initialize(PulsarService pulsarService) throws Exception { } } + @Override + public void onFilter(ServletRequest request, ServletResponse response, FilterChain chain) + throws ServletException, IOException { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.onFilter(request, response, chain); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); + } + } + @Override public void close() { final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index 5f5e260890a02..7eb1f2fae09b6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -41,6 +41,8 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.intercept.BrokerInterceptors; import org.apache.pulsar.common.util.PulsarSslConfiguration; import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.jetty.tls.JettySslContextFactory; @@ -258,7 +260,17 @@ private static class FilterInitializer { // Enable PreInterceptFilter only when interceptors are enabled filterHolders.add( new FilterHolder(new PreInterceptFilter(pulsarService.getBrokerInterceptor(), handler))); - filterHolders.add(new FilterHolder(new ProcessHandlerFilter(pulsarService.getBrokerInterceptor()))); + // The `ProcessHandlerFilter` is used to overwrite `doFilter` method, which cannot be called multiple + // times inside one `Filter`, so we cannot use one `ProcessHandlerFilter` with a `BrokerInterceptors` to + // hold all interceptors, instead we need to create a `ProcessHandlerFilter` for each `interceptor`. + if (pulsarService.getBrokerInterceptor() instanceof BrokerInterceptors) { + for (BrokerInterceptor interceptor: ((BrokerInterceptors) pulsarService.getBrokerInterceptor()) + .getInterceptors().values()) { + filterHolders.add(new FilterHolder(new ProcessHandlerFilter(interceptor))); + } + } else { + filterHolders.add(new FilterHolder(new ProcessHandlerFilter(pulsarService.getBrokerInterceptor()))); + } } if (config.isAuthenticationEnabled()) { diff --git a/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/LoggingBrokerInterceptor.java b/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/LoggingBrokerInterceptor.java index 992c6dd69a6b2..7e46ba18492d2 100644 --- a/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/LoggingBrokerInterceptor.java +++ b/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/LoggingBrokerInterceptor.java @@ -19,8 +19,10 @@ package org.apache.pulsar.tests.integration.plugins; import io.netty.buffer.ByteBuf; +import java.io.IOException; import java.util.Map; import javax.servlet.FilterChain; +import javax.servlet.ServletException; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; import org.apache.bookkeeper.mledger.Entry; @@ -122,7 +124,9 @@ public void txnEnded(String txnID, long txnAction) { } @Override - public void onFilter(ServletRequest request, ServletResponse response, FilterChain chain) { + public void onFilter(ServletRequest request, ServletResponse response, FilterChain chain) + throws ServletException, IOException { log.info("onFilter"); + chain.doFilter(request, response); } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/plugins/TestBrokerInterceptors.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/plugins/TestBrokerInterceptors.java index 98000c6f40636..b39339969e52c 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/plugins/TestBrokerInterceptors.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/plugins/TestBrokerInterceptors.java @@ -96,6 +96,7 @@ public void test(Supplier serviceUrlSupplier) throws Exception { "consumerCreated", "messageProduced", "beforeSendMessage: OK", + "onFilter", }) { assertTrue(log.contains("LoggingBrokerInterceptor - " + line), "Log did not contain line '" + line + "'"); } From 04cec0fca7930f0f800fef4119d7bd0de6097da6 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Thu, 5 Dec 2024 16:55:38 +0800 Subject: [PATCH 182/327] [improve][pip] PIP-393: Improve performance of Negative Acknowledgement (#23601) --- pip/pip-393.md | 226 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 226 insertions(+) create mode 100644 pip/pip-393.md diff --git a/pip/pip-393.md b/pip/pip-393.md new file mode 100644 index 0000000000000..646c2beb5fe40 --- /dev/null +++ b/pip/pip-393.md @@ -0,0 +1,226 @@ + +# PIP-393: Improve performance of Negative Acknowledgement + +# Background knowledge + +Negative Acknowledgement is a feature in Pulsar that allows consumers to trigger the redelivery +of a message after some time when they fail to process it. When user calls `negativeAcknowledge` method, +`NegativeAcksTracker` in `ConsumerImpl` will add an entry into the map `NegativeAcksTracker.nackedMessages`, +mapping the message ID to the redelivery time. When the redelivery time comes, `NegativeAcksTracker` will +send a redelivery request to the broker to redeliver the message. + +# Motivation + +There are many issues with the current implementation of Negative Acknowledgement in Pulsar: +- the memory occupation is high. +- the code execution efficiency is low. +- the redelivery time is not accurate. +- multiple negative ack for messages in the same entry(batch) will interfere with each other. +All of these problem is severe and need to be solved. + +## Memory occupation is high +After the improvement of https://github.com/apache/pulsar/pull/23582, we have reduce half more memory occupation +of `NegativeAcksTracker` by replacing `HashMap` with `ConcurrentLongLongPairHashMap`. With 1 million entry, the memory +occupation decrease from 178MB to 64MB. With 10 million entry, the memory occupation decrease from 1132MB to 512MB. +The average memory occupation of each entry decrease from 1132MB/10000000=118byte to 512MB/10000000=53byte. + +But it is not enough. Assuming that we negative ack message 10k/s, assigning 1h redelivery delay for each message, +the memory occupation of `NegativeAcksTracker` will be `3600*10000*53/1024/1024/1024=1.77GB`, if the delay is 5h, +the required memory is `3600*10000*53/1024/1024/1024*5=8.88GB`, which increase too fast. + +## Code execution efficiency is low +Currently, each time the timer task is triggered, it will iterate all the entries in `NegativeAcksTracker.nackedMessages`, +which is unnecessary. We can sort entries by timestamp and only iterate the entries that need to be redelivered. + +## Redelivery time is not accurate +Currently, the redelivery check time is controlled by the `timerIntervalNanos`, which is 1/3 of the `negativeAckRedeliveryDelay`. +That means, if the `negativeAckRedeliveryDelay` is 1h, check task will be started every 20min, the deviation of the redelivery +time is 20min, which is unacceptable. + +## Multiple negative ack for messages in the same entry(batch) will interfere with each other +Currently, `NegativeAcksTracker#nackedMessages` map `(ledgerId, entryId)` to `timestamp`, which means multiple nacks from messages +in the same batch share single one timestamp. +If we let msg1 redelivered 10s later, then let msg2 redelivered 20s later, these two messages are delivered 20s later together. +msg1 will not be redelivered 10s later as the timestamp recorded in `NegativeAcksTracker#nackedMessages` is overrode by the second +nack call. + + +# Goals + +Refactor the `NegativeAcksTracker` to solve the above problems. + +To avoid interation of all entries in `NegativeAcksTracker.nackedMessages`, we use a sorted map to store the entries. +To reduce memory occupation, we use util class provided by fastutil(https://fastutil.di.unimi.it/docs/), and design +a new algorithm to store the entries, reduce the memory occupation to even 1% less than the current implementation. +(the actual effect rely on the configuration and the throughput). + +# Detailed Design + +## Design & Implementation Details + +### New Data Structure +Use following data structure to store the entries: +```java +Long2ObjectSortedMap> nackedMessages = new Long2ObjectAVLTreeMap<>(); +``` +mapping `timestamp -> ledgerId -> entryId`. +We need to sort timestamp in ascending order, so we use a sorted map to map timestamp to `ledgerId -> entryId` map. +As there will many entries in the map, we use `Long2ObjectAVLTreeMap` instead of `Long2ObjectRBTreeMap`. +As for the inner map, we use `Long2ObjectMap` to map `ledgerId` to `entryId` because we don't need to keep the order of `ledgerId`. +`Long2ObjectOpenHashMap` will be satisfied. +All entry id for the same ledger id will be stored in a bit set, as we only care about the existence of the entry id. + + +### TimeStamp Bucket +Timestamp in ms is used as the key of the map. As most of the use cases don't require that the precision of the delay time is 1ms, +we can make the timestamp bucketed, that is, we can trim the lower bit of the timestamp to map the timestamp to a bucket. +For example, if we trim the lower 1 bit of the timestamp, the timestamp 0b1000 and 0b1001 will be mapped to the same bucket 0b1000. +Then all messages in the same bucket will be redelivered at the same time. +If user can accept 1024ms deviation of the redelivery time, we can trim the lower 10 bits of the timestamp, which can group a lot +entries into the same bucket and reduce the memory occupation. + +following code snippet will be helpful to understand the design: +```java + static long trimLowerBit(long timestamp, int bits) { + return timestamp & (-1L << bits); + } +``` + +```java +Long2ObjectSortedMap> map = new Long2ObjectAVLTreeMap<>(); +Long2ObjectMap ledgerMap = new Long2ObjectOpenHashMap<>(); +LongSet entrySet = new LongOpenHashSet(); +entrySet.add(entryId); +ledgerMap.put(ledgerId, entrySet); +map.put(timestamp, ledgerMap); +``` + +### Configuration + +Add a new configuration `negativeAckPrecisionBitCnt` to control the precision of the redelivery time. +``` +@ApiModelProperty( + name = "negativeAckPrecisionBitCnt", + value = "The redelivery time precision bit count. The lower bits of the redelivery time will be\n" + + "trimmed to reduce the memory occupation. The default value is 8, which means the redelivery time\n" + + "will be bucketed by 256ms. In worst cases, the redelivery time will be 512ms earlier(no later)\n" + + "than the expected time. If the value is 0, the redelivery time will be accurate to ms.". + ) + private long negativeAckPrecisionBitCnt = 8; +``` +The higher the value, the more entries will be grouped into the same bucket, the less memory occupation, the less accurate the redelivery time. +Default value is 8, which means the redelivery time will be bucketed by 256ms. In worst cases, the redelivery time will be 512ms earlier(no later) +than the expected time. + + +## Space complexity analysis +### Space complexity of `ConcurrentLongLongPairHashMap` +Before analyzing the new data structure, we need to know how much space it take before this pip. + +We need to store 4 long field for `(ledgerId, entryId, partitionIndex, timestamp)` for each entry, which takes `4*8=32byte`. +As `ConcurrentLongLongPairHashMap` use open hash addressing and linear probe to handle hash conflict, there are some +redundant spaces to avoid high conflict rate. There are two configurations that control how much redundant space to reserver: +`fill factor` and `idle factor`. When the space utility rate soar high to `fill factor`, the size of backing array will +be double, when the space utility rate reduce to `idle factor`, the size of backing array will reduce by half. + +The default value of `fill factor` is 0.66, `idle factor` is 0.15, which means the min space occupation of +`ConcurrentLongLongPairHashMap` is `32/0.66N byte = 48N byte`, the max space occupation is `32/0.15N byte=213N byte`, +where N is the number of entries. + +In the experiment showed in the PR, there are 1 million entries in the map, taking up `32*1000000/1024/1024byte=30MB`, +the space utility rate is 30/64=0.46, in the range of `[0.15, 0.66]`. + + +### Space complexity of the new data structure +The space used by new data structure is related to several factors: `message rate`, `the time deviation user accepted`, +`the max entries written in one ledger`. +- Pulsar conf `managedLedgerMaxEntriesPerLedger=50000` determine the max entries can be written into one ledger, +we use the default value to analyze. +- `the time deviation user accepted`: when user accept 1024ms delivery time deviation, we can trim the lower 10 bit +of the timestamp in ms, which can bucket 1024 timestamp. + +Following we will analyze the space used by one bucket, and calculate the average space used by one entry. + +Assuming that the message rate is `x msg/ms`, and we trim `y bit` of the timestamp, one bucket will contains `2**x` ms, and +`M=2**x*y` msgs. +- For one single bucket, we only need to store one timestamp, which takes `8byte`. +- Then, we need to store the ledgerId, when M is greater than 5w(`managedLedgerMaxEntriesPerLedger`), the ledger will switch. +There are `L=ceil(M/50000)` ledgers, which take `8*L` byte. +- Further, we analyze how much space the entry id takes. As there are `L=ceil(M/50000)` ledgers, there will be `L` bitmap to store, +which take `L*size(bitmap)`. The total space consumed by new data structure is `8byte + 8L byte + L*size(bitmap)`. + +As the `size(bitmap)` is far more greater than `8byte`, we can ignore the first two items. Then we get the formular of space +consumed **one bucket**: `D=L*size(bitmap)=ceil(M/50000)*size(bitmap)`. + +Entry id is stored in a `Roaring64Bitmap`, for simplicity we can replace it with `RoaringBitmap`, as the max entry id is 49999, +which is smaller than `4294967296 (2 * Integer.MAX_VALUE)`(the max value can be stored in `RoaringBitmap`). The space consume +by `RoaringBitmap` depends on how many elements it contains, when the size of bitmap < 4096, the space is `4N byte`, +when the size of bitmap > 4096, the consumed space is a fixed value `8KB`. + +Then we get the final result: +- when M>50000, `D = ceil(M/50000)*size(bitmap) ~= M/50000 * 8KB = M/50000 * 8 * 1024 byte = 0.163M byte`, +each entry takes `0.163byte` by average. +- when 409650000, `0.163N byte`. + - when 4096 +* Mailing List discussion thread: https://lists.apache.org/thread/yojl7ylk7cyjxktq3cn8849hvmyv0fg8 +* Mailing List voting thread: https://lists.apache.org/thread/hyc1r2s9chowdhck53lq07tznopt50dy From 28e47fa99dcb080385cfa567b844e74ab0cd85b1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 6 Dec 2024 10:14:06 -0800 Subject: [PATCH 183/327] [improve][io] Bump io.lettuce:lettuce-core from 5.0.2.RELEASE to 6.5.1.RELEASE in /pulsar-io/redis (#23685) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- pulsar-io/redis/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-io/redis/pom.xml b/pulsar-io/redis/pom.xml index 61830e8157bcf..0de19ce3d67a1 100644 --- a/pulsar-io/redis/pom.xml +++ b/pulsar-io/redis/pom.xml @@ -56,7 +56,7 @@ io.lettuce lettuce-core - 5.0.2.RELEASE + 6.5.1.RELEASE com.google.guava From 85b3d5476ed15ee4e5105139b5e75dc1e926f693 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Sat, 7 Dec 2024 06:27:00 +0800 Subject: [PATCH 184/327] [fix][cli] Fix set topic retention policy failed (#23688) --- .../src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 955d6e13e1d04..8dd6b664462ea 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -1852,7 +1852,7 @@ private class SetRetention extends CliCommand { + "-t 120 will set retention to 2 minutes. " + "0 means no retention and -1 means infinite time retention.", required = true, converter = TimeUnitToSecondsConverter.class) - private Integer retentionTimeInSec; + private Long retentionTimeInSec; @Option(names = { "--size", "-s" }, description = "Retention size limit with optional size unit suffix. " + "For example, 4096, 10M, 16G, 3T. The size unit suffix character can be k/K, m/M, g/G, or t/T. " From e6f421e670792e83594fe6fee5e92f0a586e7a9e Mon Sep 17 00:00:00 2001 From: ZachChuba <49295341+ZachChuba@users.noreply.github.com> Date: Fri, 6 Dec 2024 18:22:18 -0500 Subject: [PATCH 185/327] [fix][sec] Bump commons-io version to 2.18.0 (#23684) Co-authored-by: Zach Chuba --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- distribution/shell/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 21422a41dcef9..ae7ffaa7fc9e3 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -285,7 +285,7 @@ The Apache Software License, Version 2.0 - commons-cli-commons-cli-1.5.0.jar - commons-codec-commons-codec-1.15.jar - commons-configuration-commons-configuration-1.10.jar - - commons-io-commons-io-2.14.0.jar + - commons-io-commons-io-2.18.0.jar - commons-lang-commons-lang-2.6.jar - commons-logging-commons-logging-1.1.1.jar - org.apache.commons-commons-collections4-4.4.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index a21c272f91b1d..0b3809431df1c 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -340,7 +340,7 @@ The Apache Software License, Version 2.0 * Apache Commons - commons-codec-1.15.jar - commons-configuration-1.10.jar - - commons-io-2.14.0.jar + - commons-io-2.18.0.jar - commons-lang-2.6.jar - commons-logging-1.2.jar - commons-lang3-3.11.jar diff --git a/pom.xml b/pom.xml index 54744a253bb2b..34ba8e909d4ea 100644 --- a/pom.xml +++ b/pom.xml @@ -218,7 +218,7 @@ flexible messaging model and an intuitive client API. 2.12.1 3.11 1.10 - 2.14.0 + 2.18.0 1.15 2.1.6 2.1.9 From fa8aa9ef7056756bcb4cd04d140b5c93a28a045f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 7 Dec 2024 13:11:11 +0200 Subject: [PATCH 186/327] [fix][cli] Fix set-retention with >2GB size value for topic policy (#23689) --- .../pulsar/admin/cli/CmdTopicPolicies.java | 9 ++-- .../apache/pulsar/admin/cli/CmdTopics.java | 8 ++-- .../admin/cli/CmdTopicPoliciesTest.java | 47 +++++++++++++++++++ .../pulsar/admin/cli/TestCmdNamespaces.java | 5 +- .../pulsar/admin/cli/TestCmdTopics.java | 7 +++ 5 files changed, 65 insertions(+), 11 deletions(-) create mode 100644 pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/CmdTopicPoliciesTest.java diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java index 10850d107edf5..9a5714cb58ce0 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java @@ -29,7 +29,6 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.cli.converters.picocli.ByteUnitToIntegerConverter; import org.apache.pulsar.cli.converters.picocli.ByteUnitToLongConverter; import org.apache.pulsar.cli.converters.picocli.TimeUnitToMillisConverter; import org.apache.pulsar.cli.converters.picocli.TimeUnitToSecondsConverter; @@ -546,8 +545,8 @@ private class SetRetention extends CliCommand { + "For example, 4096, 10M, 16G, 3T. The size unit suffix character can be k/K, m/M, g/G, or t/T. " + "If the size unit suffix is not specified, the default unit is bytes. " + "0 or less than 1MB means no retention and -1 means infinite size retention", required = true, - converter = ByteUnitToIntegerConverter.class) - private Integer sizeLimit; + converter = ByteUnitToLongConverter.class) + private Long sizeLimit; @Option(names = { "--global", "-g" }, description = "Whether to set this policy globally. " + "If set to true, the policy is replicated to other clusters asynchronously, " @@ -560,8 +559,8 @@ void run() throws PulsarAdminException { final int retentionTimeInMin = retentionTimeInSec != -1 ? (int) TimeUnit.SECONDS.toMinutes(retentionTimeInSec) : retentionTimeInSec.intValue(); - final int retentionSizeInMB = sizeLimit != -1 - ? (int) (sizeLimit / (1024 * 1024)) + final long retentionSizeInMB = sizeLimit != -1 + ? (sizeLimit / (1024 * 1024)) : sizeLimit; getTopicPolicies(isGlobal).setRetention(persistentTopic, new RetentionPolicies(retentionTimeInMin, retentionSizeInMB)); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 8dd6b664462ea..22073b1a89dc9 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -1858,8 +1858,8 @@ private class SetRetention extends CliCommand { + "For example, 4096, 10M, 16G, 3T. The size unit suffix character can be k/K, m/M, g/G, or t/T. " + "If the size unit suffix is not specified, the default unit is bytes. " + "0 or less than 1MB means no retention and -1 means infinite size retention", required = true, - converter = ByteUnitToIntegerConverter.class) - private Integer sizeLimit; + converter = ByteUnitToLongConverter.class) + private Long sizeLimit; @Override void run() throws PulsarAdminException { @@ -1867,8 +1867,8 @@ void run() throws PulsarAdminException { final int retentionTimeInMin = retentionTimeInSec != -1 ? (int) TimeUnit.SECONDS.toMinutes(retentionTimeInSec) : retentionTimeInSec.intValue(); - final int retentionSizeInMB = sizeLimit != -1 - ? (int) (sizeLimit / (1024 * 1024)) + final long retentionSizeInMB = sizeLimit != -1 + ? (sizeLimit / (1024 * 1024)) : sizeLimit; getTopics().setRetention(persistentTopic, new RetentionPolicies(retentionTimeInMin, retentionSizeInMB)); } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/CmdTopicPoliciesTest.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/CmdTopicPoliciesTest.java new file mode 100644 index 0000000000000..59d3be3901935 --- /dev/null +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/CmdTopicPoliciesTest.java @@ -0,0 +1,47 @@ +/* + * 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.admin.cli; + +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.TopicPolicies; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.testng.annotations.Test; + +public class CmdTopicPoliciesTest { + + @Test + public void testSetRetentionCmd() throws Exception { + TopicPolicies topicPolicies = mock(TopicPolicies.class); + + PulsarAdmin admin = mock(PulsarAdmin.class); + when(admin.topicPolicies(anyBoolean())).thenReturn(topicPolicies); + + CmdTopicPolicies cmd = new CmdTopicPolicies(() -> admin); + + cmd.run("set-retention public/default/topic -s 2T -t 200d".split("\\s+")); + + verify(topicPolicies, times(1)).setRetention("persistent://public/default/topic", + new RetentionPolicies(200 * 24 * 60, 2 * 1024 * 1024)); + } +} \ No newline at end of file diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdNamespaces.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdNamespaces.java index f9ce84411c6c0..4ed0880d29edc 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdNamespaces.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdNamespaces.java @@ -46,7 +46,8 @@ public void testSetRetentionCmd() throws Exception { CmdNamespaces cmd = new CmdNamespaces(() -> admin); - cmd.run("set-retention public/default -s 2T -t 2h".split("\\s+")); - verify(namespaces, times(1)).setRetention("public/default", new RetentionPolicies(120, 2 * 1024 * 1024)); + cmd.run("set-retention public/default -s 2T -t 200d".split("\\s+")); + verify(namespaces, times(1)).setRetention("public/default", + new RetentionPolicies(200 * 24 * 60, 2 * 1024 * 1024)); } } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java index fc98b14392c3e..bd926edc5a808 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java @@ -50,6 +50,7 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.LedgerInfo; +import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -260,4 +261,10 @@ public void testRunDeleteTopicsFromFileWithException() throws PulsarAdminExcepti mockTopics = mock(Topics.class); } + @Test + public void testSetRetentionCmd() throws Exception { + cmdTopics.run("set-retention public/default/topic -s 2T -t 200d".split("\\s+")); + verify(mockTopics, times(1)).setRetention("persistent://public/default/topic", + new RetentionPolicies(200 * 24 * 60, 2 * 1024 * 1024)); + } } From 24c337f7a586e30a244739a4100697534fbd9f37 Mon Sep 17 00:00:00 2001 From: ran Date: Sun, 8 Dec 2024 18:28:55 +0800 Subject: [PATCH 187/327] [fix][broker] Catch exception for entry payload interceptor processor (#23683) --- .../bookkeeper/mledger/impl/OpAddEntry.java | 11 ++++- .../MangedLedgerInterceptorImplTest.java | 49 +++++++++++++++++++ 2 files changed, 58 insertions(+), 2 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java index 036ce9223e89d..2c00fd2cdf715 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java @@ -139,8 +139,15 @@ public void initiate() { lastInitTime = System.nanoTime(); if (ml.getManagedLedgerInterceptor() != null) { long originalDataLen = data.readableBytes(); - payloadProcessorHandle = ml.getManagedLedgerInterceptor() - .processPayloadBeforeLedgerWrite(this.getCtx(), duplicateBuffer); + try { + payloadProcessorHandle = ml.getManagedLedgerInterceptor() + .processPayloadBeforeLedgerWrite(this.getCtx(), duplicateBuffer); + } catch (Exception e) { + ReferenceCountUtil.safeRelease(duplicateBuffer); + log.error("[{}] Error processing payload before ledger write", ml.getName(), e); + this.failed(new ManagedLedgerException.ManagedLedgerInterceptException(e)); + return; + } if (payloadProcessorHandle != null) { duplicateBuffer = payloadProcessorHandle.getProcessedPayload(); // If data len of entry changes, correct "dataLength" and "currentLedgerSize". diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java index 74a88382b0e0e..26b2d52c194ff 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java @@ -21,9 +21,13 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.fail; + import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -431,4 +435,49 @@ public boolean test(@Nullable Entry entry) { } } + @Test(timeOut = 3000) + public void testManagedLedgerPayloadInputProcessorFailure() throws Exception { + var config = new ManagedLedgerConfig(); + final String failureMsg = "failed to process input payload"; + config.setManagedLedgerInterceptor(new ManagedLedgerInterceptorImpl( + Collections.emptySet(), Set.of(new ManagedLedgerPayloadProcessor() { + @Override + public Processor inputProcessor() { + return new Processor() { + @Override + public ByteBuf process(Object contextObj, ByteBuf inputPayload) { + throw new RuntimeException(failureMsg); + } + + @Override + public void release(ByteBuf processedPayload) { + // no-op + fail("the release method can't be reached"); + } + }; + } + }))); + + var ledger = factory.open("testManagedLedgerPayloadProcessorFailure", config); + var countDownLatch = new CountDownLatch(1); + var expectedException = new ArrayList(); + ledger.asyncAddEntry("test".getBytes(), 1, 1, new AsyncCallbacks.AddEntryCallback() { + @Override + public void addComplete(Position position, ByteBuf entryData, Object ctx) { + entryData.release(); + countDownLatch.countDown(); + } + + @Override + public void addFailed(ManagedLedgerException exception, Object ctx) { + // expected + expectedException.add(exception); + countDownLatch.countDown(); + } + }, null); + countDownLatch.await(); + assertEquals(expectedException.size(), 1); + assertEquals(expectedException.get(0).getCause().getMessage(), failureMsg); + } + } From 0a2ffe4743799dc253814d49fa3fd29b933444ac Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Sun, 8 Dec 2024 22:26:51 +0800 Subject: [PATCH 188/327] [fix][admin] Listen partitioned topic creation event (#23680) Signed-off-by: Zixuan Liu --- .../pulsar/broker/admin/AdminResource.java | 15 ++++++++++++ .../broker/TopicEventsListenerTest.java | 23 ++++++++++++++----- 2 files changed, 32 insertions(+), 6 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 45772dc279bab..4d890a3d5db4d 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 @@ -45,6 +45,8 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.broker.resources.ClusterResources; +import org.apache.pulsar.broker.service.TopicEventsListener.EventStage; +import org.apache.pulsar.broker.service.TopicEventsListener.TopicEvent; import org.apache.pulsar.broker.service.TopicPoliciesService; import org.apache.pulsar.broker.service.plugin.InvalidEntryFilterException; import org.apache.pulsar.broker.web.PulsarWebResource; @@ -166,6 +168,10 @@ public CompletableFuture validatePoliciesReadOnlyAccessAsync() { protected CompletableFuture tryCreatePartitionsAsync(int numPartitions) { if (!topicName.isPersistent()) { + for (int i = 0; i < numPartitions; i++) { + pulsar().getBrokerService().getTopicEventsDispatcher() + .notify(topicName.getPartition(i).toString(), TopicEvent.CREATE, EventStage.SUCCESS); + } return CompletableFuture.completedFuture(null); } List> futures = new ArrayList<>(numPartitions); @@ -201,6 +207,8 @@ private CompletableFuture tryCreatePartitionAsync(final int partition) { } return null; }); + pulsar().getBrokerService().getTopicEventsDispatcher() + .notifyOnCompletion(result, topicName.getPartition(partition).toString(), TopicEvent.CREATE); return result; } @@ -594,6 +602,13 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n throw new RestException(Status.CONFLICT, "This topic already exists"); } }) + .thenRun(() -> { + for (int i = 0; i < numPartitions; i++) { + pulsar().getBrokerService().getTopicEventsDispatcher() + .notify(topicName.getPartition(i).toString(), TopicEvent.CREATE, + EventStage.BEFORE); + } + }) .thenCompose(__ -> provisionPartitionedTopicPath(numPartitions, createLocalTopicOnly, properties)) .thenCompose(__ -> tryCreatePartitionsAsync(numPartitions)) .thenRun(() -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/TopicEventsListenerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/TopicEventsListenerTest.java index ceb3c1d0d9335..152b4aeeeb213 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/TopicEventsListenerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/TopicEventsListenerTest.java @@ -265,12 +265,23 @@ public void testTopicAutoGC(String topicTypePersistence, String topicTypePartiti private void createTopicAndVerifyEvents(String topicDomain, String topicTypePartitioned, String topicName) throws Exception { final String[] expectedEvents; if (topicDomain.equalsIgnoreCase("persistent") || topicTypePartitioned.equals("partitioned")) { - expectedEvents = new String[]{ - "LOAD__BEFORE", - "CREATE__BEFORE", - "CREATE__SUCCESS", - "LOAD__SUCCESS" - }; + if (topicTypePartitioned.equals("partitioned")) { + expectedEvents = new String[]{ + "CREATE__BEFORE", + "CREATE__SUCCESS", + "LOAD__BEFORE", + "CREATE__BEFORE", + "CREATE__SUCCESS", + "LOAD__SUCCESS" + }; + } else { + expectedEvents = new String[]{ + "LOAD__BEFORE", + "CREATE__BEFORE", + "CREATE__SUCCESS", + "LOAD__SUCCESS" + }; + } } else { expectedEvents = new String[]{ // Before https://github.com/apache/pulsar/pull/21995, Pulsar will skip create topic if the topic From f571aa1e6c247bea54d7a10ed508991fee1ea71b Mon Sep 17 00:00:00 2001 From: Omar Yasin Date: Wed, 11 Dec 2024 04:57:47 -0800 Subject: [PATCH 189/327] [improve][admin] Opt-out of topic-existence check (#23709) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Ómar Yasin Co-authored-by: Lari Hotari --- .../pulsar/broker/ServiceConfiguration.java | 6 +++ .../admin/impl/PersistentTopicsBase.java | 37 +++++++++++++++---- .../pulsar/broker/admin/AdminApiTest.java | 20 ++++++++++ 3 files changed, 56 insertions(+), 7 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 8b5a4ef270b0e..0f7ae00713dce 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -3535,6 +3535,12 @@ public double getLoadBalancerBandwidthOutResourceWeight() { ) private String compactionServiceFactoryClassName = "org.apache.pulsar.compaction.PulsarCompactionServiceFactory"; + @FieldContext( + category = CATEGORY_SERVER, + doc = "Opt-out of topic-existence check when setting permissions" + ) + private boolean allowAclChangesOnNonExistentTopics = false; + /**** --- KeyStore TLS config variables. --- ****/ @FieldContext( category = CATEGORY_KEYSTORE_TLS, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 6070093cc3585..9a306f6b4fff7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -206,8 +206,16 @@ protected CompletableFuture> internalGetPartitionedTopicListAsync() protected CompletableFuture>> internalGetPermissionsOnTopic() { // This operation should be reading from zookeeper and it should be allowed without having admin privileges - return validateAdminAccessForTenantAsync(namespaceName.getTenant()) - .thenCompose(__ -> internalCheckTopicExists(topicName)) + CompletableFuture validateAccessForTenantCf = + validateAdminAccessForTenantAsync(namespaceName.getTenant()); + + var checkIfTopicExists = !pulsar().getConfiguration().isAllowAclChangesOnNonExistentTopics(); + if (checkIfTopicExists) { + validateAccessForTenantCf = validateAccessForTenantCf + .thenCompose(__ -> internalCheckTopicExists(topicName)); + } + + return validateAccessForTenantCf .thenCompose(__ -> getAuthorizationService().getPermissionsAsync(topicName)); } @@ -258,9 +266,16 @@ private CompletableFuture grantPermissionsAsync(TopicName topicUri, String protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse, String role, Set actions) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges - validateAdminAccessForTenantAsync(namespaceName.getTenant()) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) - .thenCompose(__ -> internalCheckTopicExists(topicName)) + CompletableFuture validateAccessForTenantCf = validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()); + + var checkIfTopicExists = !pulsar().getConfiguration().isAllowAclChangesOnNonExistentTopics(); + if (checkIfTopicExists) { + validateAccessForTenantCf = validateAccessForTenantCf + .thenCompose(__ -> internalCheckTopicExists(topicName)); + } + + validateAccessForTenantCf .thenCompose(unused1 -> grantPermissionsAsync(topicName, role, actions)) .thenAccept(unused -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -273,8 +288,16 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges - validateAdminAccessForTenantAsync(namespaceName.getTenant()) - .thenCompose(__ -> internalCheckTopicExists(topicName)) + CompletableFuture validateAccessForTenantCf = + validateAdminAccessForTenantAsync(namespaceName.getTenant()); + + var checkIfTopicExists = !pulsar().getConfiguration().isAllowAclChangesOnNonExistentTopics(); + if (checkIfTopicExists) { + validateAccessForTenantCf = validateAccessForTenantCf + .thenCompose(__ -> internalCheckTopicExists(topicName)); + } + + validateAccessForTenantCf .thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, true, false) .thenCompose(metadata -> { int numPartitions = metadata.partitions; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 70c2b343ec584..cea43cc9345d8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.admin; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -3668,4 +3669,23 @@ public void testPermissions() { assertThrows(NotFoundException.class, () -> admin.topics().grantPermission(topic, subject, Set.of(AuthAction.produce))); assertThrows(NotFoundException.class, () -> admin.topics().revokePermissions(topic, subject)); } + + @Test + @SneakyThrows + public void testPermissionsAllowAclChangesOnNonExistentTopics() { + pulsar.getConfiguration().setAllowAclChangesOnNonExistentTopics(true); + try { + String namespace = "prop-xyz/ns1/"; + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://" + namespace + random; + final String subject = UUID.randomUUID().toString(); + admin.topics().grantPermission(topic, subject, Set.of(AuthAction.produce)); + assertThat(admin.topics().getPermissions(topic).get(subject)).containsExactly(AuthAction.produce); + admin.topics().revokePermissions(topic, subject); + assertThat(admin.topics().getPermissions(topic).get(subject)).isNullOrEmpty(); + } finally { + // reset config + pulsar.getConfiguration().setAllowAclChangesOnNonExistentTopics(false); + } + } } From c52af1d5c733ec05b40fc72e63a202f20d25603b Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Thu, 12 Dec 2024 21:32:20 +0800 Subject: [PATCH 190/327] [improve][client] Print consumer stats log if prefetched messages are not zero (#23698) --- .../apache/pulsar/client/impl/ConsumerStatsRecorderImpl.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerStatsRecorderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerStatsRecorderImpl.java index 8dfc0af8e1d93..5cbbcc4429883 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerStatsRecorderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerStatsRecorderImpl.java @@ -146,15 +146,16 @@ private void init(ConsumerConfigurationData conf) { receivedMsgsRate = currentNumMsgsReceived / elapsed; receivedBytesRate = currentNumBytesReceived / elapsed; + int prefetchQueueSize = consumerImpl.incomingMessages.size(); if ((currentNumMsgsReceived | currentNumBytesReceived | currentNumReceiveFailed | currentNumAcksSent - | currentNumAcksFailed) != 0) { + | currentNumAcksFailed | prefetchQueueSize) != 0) { log.info( "[{}] [{}] [{}] Prefetched messages: {} --- " + "Consume throughput received: {} msgs/s --- {} Mbit/s --- " + "Ack sent rate: {} ack/s --- " + "Failed messages: {} --- batch messages: {} ---" + "Failed acks: {}", consumerImpl.getTopic(), consumerImpl.getSubscription(), consumerImpl.consumerName, - consumerImpl.incomingMessages.size(), THROUGHPUT_FORMAT.format(receivedMsgsRate), + prefetchQueueSize, THROUGHPUT_FORMAT.format(receivedMsgsRate), THROUGHPUT_FORMAT.format(receivedBytesRate * 8 / 1024 / 1024), THROUGHPUT_FORMAT.format(currentNumAcksSent / elapsed), currentNumReceiveFailed, currentNumBatchReceiveFailed, currentNumAcksFailed); From 39f4ccdc2c9fe60e9a35bcced81bf915fa3df294 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 13 Dec 2024 09:42:17 +0800 Subject: [PATCH 191/327] [fix] [broker] Fix config replicationStartAt does not work when set it to earliest (#23719) Co-authored-by: Lari Hotari --- .../pulsar/broker/service/persistent/PersistentTopic.java | 8 ++++++-- .../pulsar/broker/service/OneWayReplicatorTest.java | 4 ++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index eb48ceee72d76..0b2c9d8c7bcb6 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 @@ -2066,9 +2066,13 @@ CompletableFuture startReplicator(String remoteCluster) { final CompletableFuture future = new CompletableFuture<>(); String name = PersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); + String replicationStartAt = getBrokerService().getPulsar().getConfiguration().getReplicationStartAt(); final InitialPosition initialPosition; - if (MessageId.earliest.toString() - .equalsIgnoreCase(getBrokerService().getPulsar().getConfiguration().getReplicationStartAt())) { + // "MessageId.earliest.toString()" is "-1:-1:-1", which is not suggested, just guarantee compatibility with the + // previous version. + // "InitialPosition.Earliest.name()" is "Earliest", which is suggested. + if (MessageId.earliest.toString().equalsIgnoreCase(replicationStartAt) + || InitialPosition.Earliest.name().equalsIgnoreCase(replicationStartAt)) { initialPosition = InitialPosition.Earliest; } else { initialPosition = InitialPosition.Latest; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index a8f8d7ecbbd47..d3356a6069553 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -1023,9 +1023,9 @@ public void testConfigReplicationStartAt() throws Exception { disableReplication(topic1); // 2.Update config: start at "earliest". - admin1.brokers().updateDynamicConfiguration("replicationStartAt", MessageId.earliest.toString()); + admin1.brokers().updateDynamicConfiguration("replicationStartAt", "earliest"); Awaitility.await().untilAsserted(() -> { - pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("earliest"); + assertEquals(pulsar1.getConfiguration().getReplicationStartAt(), "earliest"); }); final String topic2 = BrokerTestUtil.newUniqueName("persistent://" + ns1 + "/tp_"); From 8e80f88cd46ad041b87773d49c5ce4420df95b9a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 13 Dec 2024 11:36:09 +0800 Subject: [PATCH 192/327] [fix] [broker] fix NPE when calculating a topic's backlogQuota (#23720) --- .../service/persistent/PersistentTopic.java | 6 +- .../PersistentTopicProtectedMethodsTest.java | 114 ++++++++++++++++++ 2 files changed, 118 insertions(+), 2 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicProtectedMethodsTest.java 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 0b2c9d8c7bcb6..056fad2a005b4 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 @@ -3631,7 +3631,8 @@ public CompletableFuture checkTimeBacklogExceeded(boolean shouldUpdateO }); } - private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( + @VisibleForTesting + EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( Position markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); @@ -3650,7 +3651,8 @@ private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaC // if the mark-delete position is the last entry it means all entries for // that ledger are acknowledged - if (markDeletePosition.getEntryId() == markDeletePositionLedgerInfo.getEntries() - 1) { + if (markDeletePositionLedgerInfo != null + && (markDeletePosition.getEntryId() == markDeletePositionLedgerInfo.getEntries() - 1)) { Position positionToCheck = ledger.getNextValidPosition(markDeletePosition); positionToCheckLedgerInfo = ledger.getLedgerInfo(positionToCheck.getLedgerId()).get(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicProtectedMethodsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicProtectedMethodsTest.java new file mode 100644 index 0000000000000..1d841483ed723 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicProtectedMethodsTest.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import static org.testng.Assert.assertEquals; +import java.util.concurrent.CompletableFuture; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +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 PersistentTopicProtectedMethodsTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + protected void doInitConf() throws Exception { + this.conf.setPreciseTimeBasedBacklogQuotaCheck(true); + this.conf.setManagedLedgerMaxEntriesPerLedger(2); + this.conf.setManagedLedgerMaxLedgerRolloverTimeMinutes(10); + this.conf.setManagedLedgerMinLedgerRolloverTimeMinutes(0); + } + + /*** + * Background: the steps for checking backlog metadata are as follows. + * - Get the oldest cursor. + * - Return the result if the oldest `cursor.md` equals LAC. + * - Else, calculate the estimated backlog quota. + * + * What case been covered by this test. + * - The method `PersistentTopic.estimatedTimeBasedBacklogQuotaCheck` may get an NPE when the + * `@param position(cursor.markDeletedPositon)` equals LAC and the latest ledger has been removed by a + * `ML.trimLedgers`, which was introduced by https://github.com/apache/pulsar/pull/21816. + * - Q: The broker checked whether the oldest `cursor.md` equals LAC at step 2 above, why does it still call + * `PersistentTopic.estimatedTimeBasedBacklogQuotaCheck` with a param that equals `LAC`? + * - A: There may be some `acknowledgments` and `ML.trimLedgers` that happened between `step2 above and step 3`. + */ + @Test + public void testEstimatedTimeBasedBacklogQuotaCheckWhenNoBacklog() throws Exception { + final String tp = BrokerTestUtil.newUniqueName("public/default/tp"); + admin.topics().createNonPartitionedTopic(tp); + PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(tp, false).join().get(); + ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + Consumer c1 = pulsarClient.newConsumer().topic(tp).subscriptionName("s1").subscribe(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get("s1"); + + // Generated multi ledgers. + Producer p1 = pulsarClient.newProducer().topic(tp).create(); + byte[] content = new byte[]{1}; + for (int i = 0; i < 10; i++) { + p1.send(content); + } + + // Consume all messages. + // Trim ledgers, then the LAC relates to a ledger who has been deleted. + admin.topics().skipAllMessages(tp, "s1"); + Awaitility.await().untilAsserted(() -> { + assertEquals(cursor.getNumberOfEntriesInBacklog(true), 0); + assertEquals(cursor.getMarkDeletedPosition(), ml.getLastConfirmedEntry()); + }); + CompletableFuture completableFuture = new CompletableFuture(); + ml.trimConsumedLedgersInBackground(completableFuture); + completableFuture.join(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml.getLedgersInfo().size(), 1); + assertEquals(cursor.getNumberOfEntriesInBacklog(true), 0); + assertEquals(cursor.getMarkDeletedPosition(), ml.getLastConfirmedEntry()); + }); + + // Verify: "persistentTopic.estimatedTimeBasedBacklogQuotaCheck" will not get a NullPointerException. + Position oldestPosition = ml.getCursors().getCursorWithOldestPosition().getPosition(); + persistentTopic.estimatedTimeBasedBacklogQuotaCheck(oldestPosition); + + p1.close(); + c1.close(); + admin.topics().delete(tp, false); + } +} From 4606385f4e30392119b813326d493245a3504aac Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 13 Dec 2024 14:38:03 +0800 Subject: [PATCH 193/327] [fix][client] Fix wrong start message id when it's a chunked message id (#23713) --- .../client/impl/MessageChunkingTest.java | 25 ++++++++++++------- .../pulsar/client/impl/ConsumerImpl.java | 8 +++++- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java index 8df5a38bb461c..18ba6a5ab5b37 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java @@ -561,8 +561,12 @@ protected void customizeNewPulsarClientBuilder(ClientBuilder clientBuilder) { clientBuilder.memoryLimit(10000L, SizeUnit.BYTES); } + interface ThrowingBiConsumer { + void accept(T t, U u) throws Exception; + } + @Test - public void testSeekChunkMessages() throws PulsarClientException { + public void testSeekChunkMessages() throws Exception { log.info("-- Starting {} test --", methodName); this.conf.setMaxMessageSize(50); final int totalMessages = 5; @@ -612,14 +616,17 @@ public void testSeekChunkMessages() throws PulsarClientException { assertEquals(msgIds.get(i), msgAfterSeek.getMessageId()); } - Reader reader = pulsarClient.newReader() - .topic(topicName) - .startMessageIdInclusive() - .startMessageId(msgIds.get(1)) - .create(); - - Message readMsg = reader.readNext(5, TimeUnit.SECONDS); - assertEquals(msgIds.get(1), readMsg.getMessageId()); + ThrowingBiConsumer assertStartMessageId = (inclusive, expectedFirstMsgId) -> { + final var builder = pulsarClient.newReader().topic(topicName).startMessageId(msgIds.get(1)); + if (inclusive) { + builder.startMessageIdInclusive(); + } + @Cleanup final var reader = builder.create(); + final var readMsg = reader.readNext(5, TimeUnit.SECONDS); + assertEquals(expectedFirstMsgId, readMsg.getMessageId()); + }; + assertStartMessageId.accept(true, msgIds.get(1)); + assertStartMessageId.accept(false, msgIds.get(2)); consumer1.close(); consumer2.close(); 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 d2753856264fc..e01c6d4643b08 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 @@ -298,7 +298,13 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat this.subscriptionMode = conf.getSubscriptionMode(); if (startMessageId != null) { MessageIdAdv firstChunkMessageId = ((MessageIdAdv) startMessageId).getFirstChunkMessageId(); - this.startMessageId = (firstChunkMessageId == null) ? (MessageIdAdv) startMessageId : firstChunkMessageId; + if (conf.isResetIncludeHead() && firstChunkMessageId != null) { + // The chunk message id's ledger id and entry id are the last chunk's ledger id and entry id, when + // startMessageIdInclusive() is enabled, we need to start from the first chunk's message id + this.startMessageId = firstChunkMessageId; + } else { + this.startMessageId = (MessageIdAdv) startMessageId; + } } this.initialStartMessageId = this.startMessageId; this.startMessageRollbackDurationInSec = startMessageRollbackDurationInSec; From 51e8247e497626eee3ff83056f7ca24acf4cf57a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 13 Dec 2024 15:12:59 +0200 Subject: [PATCH 194/327] [fix][sec] Mitigate CVE-2024-53990 by disabling AsyncHttpClient CookieStore (#23725) --- .../broker/authentication/oidc/AuthenticationProviderOpenID.java | 1 + .../org/apache/pulsar/client/api/BrokerServiceLookupTest.java | 1 + .../pulsar/client/admin/internal/http/AsyncHttpConnector.java | 1 + .../org/apache/pulsar/client/impl/ControlledClusterFailover.java | 1 + .../src/main/java/org/apache/pulsar/client/impl/HttpClient.java | 1 + .../pulsar/client/impl/auth/oauth2/protocol/TokenClient.java | 1 + 6 files changed, 6 insertions(+) diff --git a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java index 38f618091333a..cdccaa3ad39b0 100644 --- a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java +++ b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java @@ -183,6 +183,7 @@ public void initialize(Context context) throws IOException { .build(); } AsyncHttpClientConfig clientConfig = new DefaultAsyncHttpClientConfig.Builder() + .setCookieStore(null) .setConnectTimeout(connectionTimeout) .setReadTimeout(readTimeout) .setSslContext(sslContext) 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 157df1185307a..07deb9007c487 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 @@ -1142,6 +1142,7 @@ public void onThrowable(Throwable t) { private AsyncHttpClient getHttpClient(String version) { DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); + confBuilder.setCookieStore(null); confBuilder.setUseProxyProperties(true); confBuilder.setFollowRedirect(true); confBuilder.setUserAgent(version); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java index b04c19b7ded45..64ba0e99cb915 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java @@ -159,6 +159,7 @@ private void configureAsyncHttpClientConfig(ClientConfigurationData conf, int co if (conf.getConnectionMaxIdleSeconds() > 0) { confBuilder.setPooledConnectionIdleTimeout(conf.getConnectionMaxIdleSeconds() * 1000); } + confBuilder.setCookieStore(null); confBuilder.setUseProxyProperties(true); confBuilder.setFollowRedirect(false); confBuilder.setRequestTimeout(conf.getRequestTimeoutMs()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index 080d328e3f02c..1674f7469affc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -85,6 +85,7 @@ private ControlledClusterFailover(ControlledClusterFailoverBuilderImpl builder) private AsyncHttpClient buildHttpClient() { DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); + confBuilder.setCookieStore(null); confBuilder.setUseProxyProperties(true); confBuilder.setFollowRedirect(true); confBuilder.setMaxRedirects(DEFAULT_MAX_REDIRECTS); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java index 53796ff7a4bf5..c53f79af12e75 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java @@ -74,6 +74,7 @@ protected HttpClient(ClientConfigurationData conf, EventLoopGroup eventLoopGroup this.serviceNameResolver.updateServiceUrl(conf.getServiceUrl()); DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); + confBuilder.setCookieStore(null); confBuilder.setUseProxyProperties(true); confBuilder.setFollowRedirect(true); confBuilder.setMaxRedirects(conf.getMaxLookupRedirects()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java index 768f6d31d18a2..da596ce0985e3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java @@ -53,6 +53,7 @@ public TokenClient(URL tokenUrl) { TokenClient(URL tokenUrl, AsyncHttpClient httpClient) { if (httpClient == null) { DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); + confBuilder.setCookieStore(null); confBuilder.setUseProxyProperties(true); confBuilder.setFollowRedirect(true); confBuilder.setConnectTimeout(DEFAULT_CONNECT_TIMEOUT_IN_SECONDS * 1000); From 0845c2163d3a2a118cc9ae3bc80ad97d5eec8a39 Mon Sep 17 00:00:00 2001 From: Yike Xiao Date: Sat, 14 Dec 2024 02:14:15 +0800 Subject: [PATCH 195/327] [fix][doc] Refine ClientBuilder#memoryLimit and ConsumerBuilder#autoScaledReceiverQueueSizeEnabled javadoc (#23687) --- .../java/org/apache/pulsar/client/api/ClientBuilder.java | 5 ++--- .../org/apache/pulsar/client/api/ConsumerBuilder.java | 9 ++++++--- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java index 73ad555165c05..74b7c084d5ada 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java @@ -439,9 +439,8 @@ ClientBuilder authentication(String authPluginClassName, Map aut ClientBuilder tlsProtocols(Set tlsProtocols); /** - * Configure a limit on the amount of direct memory that will be allocated by this client instance. - *

- * Note: at this moment this is only limiting the memory for producers. + * Configure a limit on the amount of direct memory that will be allocated by this client instance + * (default: 64 MB). *

* Setting this to 0 will disable the limit. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index 4c37b1da927a6..3ce12b7741a8f 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -884,13 +884,16 @@ public interface ConsumerBuilder extends Cloneable { /** * If this is enabled, the consumer receiver queue size is initialized as a very small value, 1 by default, - * and will double itself until it reaches the value set by {@link #receiverQueueSize(int)}, if and only if: + * and will double itself until it reaches either the value set by {@link #receiverQueueSize(int)} or the client + * memory limit set by {@link ClientBuilder#memoryLimit(long, SizeUnit)}. + * + *

The consumer receiver queue size will double if and only if: *

1) User calls receive() and there are no messages in receiver queue. *

2) The last message we put in the receiver queue took the last space available in receiver queue. * - * This is disabled by default and currentReceiverQueueSize is initialized as maxReceiverQueueSize. + *

This is disabled by default and currentReceiverQueueSize is initialized as maxReceiverQueueSize. * - * The feature should be able to reduce client memory usage. + *

The feature should be able to reduce client memory usage. * * @param enabled whether to enable AutoScaledReceiverQueueSize. */ From 0ae3f9def9c4f35d9cd5bd590a0110bc76f13cab Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Sat, 14 Dec 2024 14:49:10 +0800 Subject: [PATCH 196/327] [fix][broker] Fix the retry mechanism in `MetadataCache#readModifyUpdateOrCreate` (#23686) ## Motivation 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. ## Modification - 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 --- .../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, 135 insertions(+), 39 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 2bc042aebb308..641d4bde5a23b 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 @@ -24,6 +24,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. @@ -33,6 +34,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 @@ -57,4 +62,7 @@ public class MetadataCacheConfig { @Builder.Default private final BiConsumer>> asyncReloadConsumer = null; + @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 4c7f34aa5c16e..66c8388c83bd2 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 @@ -30,13 +30,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; @@ -58,18 +61,23 @@ 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; if (store instanceof MetadataStoreExtended) { this.storeExtended = (MetadataStoreExtended) store; @@ -77,6 +85,8 @@ public MetadataCacheImpl(MetadataStore store, MetadataSerde serde, MetadataCa this.storeExtended = null; } this.serde = serde; + this.cacheConfig = cacheConfig; + this.executor = executor; Caffeine cacheBuilder = Caffeine.newBuilder(); if (cacheConfig.getRefreshAfterWriteMillis() > 0) { @@ -321,22 +331,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 c458d0da2146a..f63aa1c036d88 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 @@ -236,21 +236,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 6992c69b7252e..ddd975e422ab8 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.EnumSet; @@ -36,6 +41,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; @@ -44,6 +51,8 @@ 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; @@ -51,6 +60,7 @@ 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; @@ -60,6 +70,7 @@ import org.apache.pulsar.metadata.api.extended.CreateOption; 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; @@ -488,32 +499,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()); - @Cleanup - 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); + cache.create(key1, value1).join(); + assertEquals(cache.get(key1).join().get().b, 1); - CompletableFuture future1 = objCache1.readModifyUpdate(key1, v -> { - return new MyClass(v.a, v.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") @@ -647,4 +700,15 @@ public void testAsyncReloadConsumer(String provider, Supplier urlSupplie refreshed.contains(value2); }); } + + @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 1113153bd33a59b5997a1473514cc63f34d54991 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 16 Dec 2024 10:11:58 +0800 Subject: [PATCH 197/327] [improve] [test] Add more test for the case that client receives a SendError, which relates to the PR #23038 (#23721) --- .../pulsar/client/impl/ClientCnxTest.java | 59 ++++++++++++++++++- 1 file changed, 57 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java index e25212e0108f8..35b89ad988cc4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java @@ -26,10 +26,14 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.service.ServerCnx; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; @@ -44,6 +48,7 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +@Slf4j @Test(groups = "broker-impl") public class ClientCnxTest extends MockedPulsarServiceBaseTest { @@ -137,6 +142,7 @@ public void testClientVersion() throws Exception { public void testCnxReceiveSendError() throws Exception { final String topicOne = "persistent://" + NAMESPACE + "/testCnxReceiveSendError-one"; final String topicTwo = "persistent://" + NAMESPACE + "/testCnxReceiveSendError-two"; + final String topicThree = "persistent://" + NAMESPACE + "/testCnxReceiveSendError-three"; PulsarClient client = PulsarClient.builder().serviceUrl(lookupUrl.toString()).connectionsPerBroker(1).build(); Producer producerOne = client.newProducer(Schema.STRING) @@ -145,22 +151,31 @@ public void testCnxReceiveSendError() throws Exception { Producer producerTwo = client.newProducer(Schema.STRING) .topic(topicTwo) .create(); + Producer producerThree = client.newProducer(Schema.STRING) + .topic(topicThree).producerName("three") + .create(); ClientCnx cnxOne = ((ProducerImpl) producerOne).getClientCnx(); ClientCnx cnxTwo = ((ProducerImpl) producerTwo).getClientCnx(); + ClientCnx cnxThree = ((ProducerImpl) producerTwo).getClientCnx(); // simulate a sending error cnxOne.handleSendError(Commands.newSendErrorCommand(((ProducerImpl) producerOne).producerId, - 10, ServerError.PersistenceError, "persistent error").getSendError()); + 10, ServerError.PersistenceError, "persistent error 1").getSendError()); + cnxThree.handleSendError(Commands.newSendErrorCommand(((ProducerImpl) producerOne).producerId, + 10, ServerError.PersistenceError, "persistent error 3").getSendError()); // two producer use the same cnx Assert.assertEquals(cnxOne, cnxTwo); + Assert.assertEquals(cnxThree, cnxTwo); // the cnx will not change try { Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> (((ProducerImpl) producerOne).getClientCnx() != null && !cnxOne.equals(((ProducerImpl) producerOne).getClientCnx())) - || !cnxTwo.equals(((ProducerImpl) producerTwo).getClientCnx())); + || (((ProducerImpl) producerThree).getClientCnx() != null + && !cnxThree.equals(((ProducerImpl) producerThree).getClientCnx())) + || !cnxTwo.equals(((ProducerImpl) producerTwo).getClientCnx())); Assert.fail(); } catch (Throwable e) { Assert.assertTrue(e instanceof ConditionTimeoutException); @@ -173,11 +188,51 @@ public void testCnxReceiveSendError() throws Exception { // producer also can send message producerOne.send("test"); producerTwo.send("test"); + producerThree.send("test"); producerTwo.close(); producerOne.close(); + producerThree.close(); client.close(); } + @Test + public void testCnxReceiveSendErrorWithMultiConnectionsPerBroker() throws Exception { + PulsarClient client = PulsarClient.builder().serviceUrl(lookupUrl.toString()) + .connectionsPerBroker(1000).build(); + + // Create a producer with customized name. + final String tp = BrokerTestUtil.newUniqueName(NAMESPACE + "/tp"); + admin.topics().createNonPartitionedTopic(tp); + ProducerImpl p = + (ProducerImpl) client.newProducer(Schema.STRING).producerName("p1").topic(tp).create(); + + // Inject a persistence error. + org.apache.pulsar.broker.service.Producer serverProducer = pulsar.getBrokerService().getTopic(tp, false) + .join().get().getProducers().values().iterator().next(); + ServerCnx serverCnx = (ServerCnx) serverProducer.getCnx(); + serverCnx.getCommandSender().sendSendError(serverProducer.getProducerId(), 1/* sequenceId */, + ServerError.PersistenceError, "mocked error"); + + // Wait for the client receives the error. + // If the client confirmed two Pings, it means the client has handled the PersistenceError we sent. + serverCnx.checkConnectionLiveness().join(); + serverCnx.checkConnectionLiveness().join(); + + try { + // Verify: the next publish will finish. + MessageId messageId = p.sendAsync("1").get(10, TimeUnit.SECONDS); + MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; + log.info("sent {}:{}", messageIdAdv.getLedgerId(), messageIdAdv.getEntryId()); + } finally { + // cleanup orphan producers. + serverCnx.ctx().close(); + // cleanup + client.close(); + p.close(); + admin.topics().delete(tp); + } + } + public void testSupportsGetPartitionedMetadataWithoutAutoCreation() throws Exception { final String topic = BrokerTestUtil.newUniqueName( "persistent://" + NAMESPACE + "/tp"); admin.topics().createNonPartitionedTopic(topic); From 8d402f446117541ba037c2608e4bfab9be7ad970 Mon Sep 17 00:00:00 2001 From: hanmz Date: Mon, 16 Dec 2024 17:24:19 +0800 Subject: [PATCH 198/327] [fix][broker] Remove delta when calculating msgRateRequiredFromUnloadedBundles and msgThroughputRequiredFromUnloadedBundles for UniformLoadShedder (#21906) --- .../loadbalance/impl/UniformLoadShedder.java | 6 +- .../impl/UniformLoadShedderTest.java | 80 +++++++++++++++++++ 2 files changed, 82 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedder.java index 78bdbc5711201..1f292c0e8faf3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedder.java @@ -154,8 +154,7 @@ public Multimap findBundlesForUnloading(final LoadData loadData, } String bundle = e.getLeft(); double bundleMsgRate = e.getRight(); - if (bundleMsgRate <= (msgRateRequiredFromUnloadedBundles.getValue() - + 1000/* delta */)) { + if (bundleMsgRate <= msgRateRequiredFromUnloadedBundles.getValue()) { log.info("Found bundle to unload with msgRate {}", bundleMsgRate); msgRateRequiredFromUnloadedBundles.add(-bundleMsgRate); selectedBundlesCache.put(msgRateOverloadedBroker.getValue(), bundle); @@ -193,8 +192,7 @@ public Multimap findBundlesForUnloading(final LoadData loadData, } String bundle = e.getLeft(); double msgThroughput = e.getRight(); - if (msgThroughput <= (msgThroughputRequiredFromUnloadedBundles.getValue() - + 1000/* delta */)) { + if (msgThroughput <= msgThroughputRequiredFromUnloadedBundles.getValue()) { log.info("Found bundle to unload with msgThroughput {}", msgThroughput); msgThroughputRequiredFromUnloadedBundles.add(-msgThroughput); selectedBundlesCache.put(msgThroughputOverloadedBroker.getValue(), bundle); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedderTest.java index 4b4042cf31a72..a49dacd27f024 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/UniformLoadShedderTest.java @@ -197,4 +197,84 @@ public void testOverloadBrokerSelect() { assertTrue(bundlesToUnload.containsKey("broker2")); } + @Test + public void testSmallTrafficShedding(){ + conf.setMinUnloadMessage(0); + conf.setMinUnloadMessageThroughput(0); + conf.setMaxUnloadPercentage(0.5); + conf.setMaxUnloadBundleNumPerShedding(-1); + + // The situation under small msgRate + int numBundles = 5; + LoadData loadData = new LoadData(); + + LocalBrokerData broker1 = new LocalBrokerData(); + LocalBrokerData broker2 = new LocalBrokerData(); + + double brokerMsgRateIn = 0; + double brokerMsgRateOut = 0; + + for (int i = 1; i <= numBundles; ++i) { + broker1.getBundles().add("bundle-" + i); + + BundleData bundle = new BundleData(); + + double msgRateIn = 50; + double msgRateOut = 50; + TimeAverageMessageData timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(msgRateIn); + timeAverageMessageData.setMsgRateOut(msgRateOut); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle-" + i, bundle); + + brokerMsgRateIn += msgRateIn; + brokerMsgRateOut += msgRateOut; + } + + broker1.setMsgRateIn(brokerMsgRateIn); + broker1.setMsgRateOut(brokerMsgRateOut); + + loadData.getBrokerData().put("broker-1", new BrokerData(broker1)); + loadData.getBrokerData().put("broker-2", new BrokerData(broker2)); + + Multimap bundlesToUnload = uniformLoadShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 2); + + + // The situation under small throughput + loadData = new LoadData(); + + broker1 = new LocalBrokerData(); + broker2 = new LocalBrokerData(); + + double brokerThroughputIn = 0; + double brokerThroughputOut = 0; + + for (int i = 1; i <= numBundles; ++i) { + broker1.getBundles().add("bundle-" + i); + + BundleData bundle = new BundleData(); + + double msgThroughputIn = 1024; + double msgThroughputOut = 1024; + TimeAverageMessageData timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgThroughputIn(msgThroughputIn); + timeAverageMessageData.setMsgThroughputOut(msgThroughputOut); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle-" + i, bundle); + + brokerThroughputIn += msgThroughputIn; + brokerThroughputOut += msgThroughputOut; + } + + broker1.setMsgThroughputIn(brokerThroughputIn); + broker1.setMsgThroughputOut(brokerThroughputOut); + + loadData.getBrokerData().put("broker-1", new BrokerData(broker1)); + loadData.getBrokerData().put("broker-2", new BrokerData(broker2)); + + bundlesToUnload = uniformLoadShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 2); + } + } From 3761dc482cf0566ad012b06a908422938bc3f391 Mon Sep 17 00:00:00 2001 From: hanmz Date: Mon, 16 Dec 2024 19:22:28 +0800 Subject: [PATCH 199/327] [fix][broker] Avoid block markDeletePosition forward when skip lost entries (#21210) Co-authored-by: Lari Hotari Co-authored-by: Lari Hotari --- .../mledger/impl/ManagedCursorImpl.java | 48 +++++++++++++++++++ .../bookkeeper/mledger/impl/OpReadEntry.java | 2 + .../mledger/impl/ManagedCursorTest.java | 47 ++++++++++++++++++ 3 files changed, 97 insertions(+) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 478c6a1b37976..022cecf8d57b5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -2890,6 +2890,54 @@ public void deleteFailed(ManagedLedgerException ex, Object ctx) { }, null); } + /** + * Manually acknowledge all entries from startPosition to endPosition. + * - Since this is an uncommon event, we focus on maintainability. So we do not modify + * {@link #individualDeletedMessages} and {@link #batchDeletedIndexes}, but call + * {@link #asyncDelete(Position, AsyncCallbacks.DeleteCallback, Object)}. + * - This method is valid regardless of the consumer ACK type. + * - If there is a consumer ack request after this event, it will also work. + */ + public void skipNonRecoverableEntries(Position startPosition, Position endPosition){ + long ledgerId = startPosition.getLedgerId(); + LedgerInfo ledgerInfo = ledger.getLedgersInfo().get(ledgerId); + if (ledgerInfo == null) { + return; + } + + long startEntryId = Math.max(0, startPosition.getEntryId()); + long endEntryId = ledgerId != endPosition.getLedgerId() ? ledgerInfo.getEntries() : endPosition.getEntryId(); + if (startEntryId >= endEntryId) { + return; + } + + lock.writeLock().lock(); + log.warn("[{}] [{}] Since these entry for ledger [{}] is lost and the autoSkipNonRecoverableData is true, " + + "these entries [{}:{}) will be auto acknowledge in subscription", + ledger.getName(), name, ledgerId, startEntryId, endEntryId); + try { + for (long i = startEntryId; i < endEntryId; i++) { + if (!individualDeletedMessages.contains(ledgerId, i)) { + asyncDelete(PositionFactory.create(ledgerId, i), new AsyncCallbacks.DeleteCallback() { + @Override + public void deleteComplete(Object ctx) { + // ignore. + } + + @Override + public void deleteFailed(ManagedLedgerException ex, Object ctx) { + // The method internalMarkDelete already handled the failure operation. We only need to + // make sure the memory state is updated. + // If the broker crashed, the non-recoverable ledger will be detected again. + } + }, null); + } + } + } finally { + lock.writeLock().unlock(); + } + } + // ////////////////////////////////////////////////// void startCreatingNewMetadataLedger() { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index 3fd7e36c433ae..a4928b44bd97d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -137,6 +137,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { updateReadPosition(nexReadPosition); if (lostLedger != null) { cursor.getManagedLedger().skipNonRecoverableLedger(lostLedger); + } else { + cursor.skipNonRecoverableEntries(readPosition, nexReadPosition); } checkReadCompletion(); } else { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index b4ab673facb26..69b74fcf8f5c1 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -4766,6 +4766,53 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { && cursorReadPosition.getEntryId() == expectReadPosition.getEntryId()); } + @Test + public void testSkipNonRecoverableEntries() throws ManagedLedgerException, InterruptedException { + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + int maxMessagePerLedger = 10; + managedLedgerConfig.setMaxEntriesPerLedger(maxMessagePerLedger); + ManagedLedger ledger = factory.open("testSkipNonRecoverableEntries", managedLedgerConfig); + ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("my-cursor"); + + Position lacPosition = ledger.getLastConfirmedEntry(); + long ledgerId = lacPosition.getLedgerId(); + assertEquals(PositionFactory.create(ledgerId, -1), cursor.getMarkDeletedPosition()); + + // Mock add 10 entry + for (int i = 0; i < 10; i++) { + ledger.addEntry(String.valueOf(i).getBytes()); + } + + // read 2 entry and delete these entries, MarkDeletedPosition move forward + List entries = cursor.readEntries(2); + for (Entry entry : entries) { + cursor.delete(entry.getPosition()); + } + assertEquals(PositionFactory.create(ledgerId, 1), cursor.getMarkDeletedPosition()); + + // read the next 6 entry and not delete, MarkDeletedPosition not move forward + entries = cursor.readEntries(6); + assertEquals(PositionFactory.create(ledgerId, 1), cursor.getMarkDeletedPosition()); + + // delete last read entry, MarkDeletedPosition not move forward + Entry lastEntry = entries.get(entries.size() - 1); + cursor.delete(lastEntry.getPosition()); + assertEquals(PositionFactory.create(ledgerId, 1), cursor.getMarkDeletedPosition()); + + // call skip entries, MarkDeletedPosition move forward + cursor.skipNonRecoverableEntries(cursor.getMarkDeletedPosition(), + PositionFactory.create(ledgerId, lastEntry.getEntryId())); + assertEquals(PositionFactory.create(ledgerId, lastEntry.getEntryId()), cursor.getMarkDeletedPosition()); + + // repeat call skip entries, MarkDeletedPosition not change + cursor.skipNonRecoverableEntries(cursor.getMarkDeletedPosition(), + PositionFactory.create(ledgerId, lastEntry.getEntryId())); + assertEquals(PositionFactory.create(ledgerId, lastEntry.getEntryId()), cursor.getMarkDeletedPosition()); + + cursor.close(); + ledger.close(); + } + @Test public void testRecoverCursorWithTerminateManagedLedger() throws Exception { String mlName = "my_test_ledger"; From 9a7269a9cf98bd63f45cc48db85b70fb191fb054 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 16 Dec 2024 13:51:28 +0200 Subject: [PATCH 200/327] [fix][sec] Upgrade async-http-client to 2.12.4 to address CVE-2024-53990 (#23732) --- distribution/server/src/assemble/LICENSE.bin.txt | 6 +++--- distribution/shell/src/assemble/LICENSE.bin.txt | 6 +++--- pom.xml | 9 +-------- pulsar-broker/pom.xml | 2 +- pulsar-client-admin-shaded/pom.xml | 2 +- pulsar-client-admin/pom.xml | 2 +- pulsar-client-all/pom.xml | 2 +- pulsar-client-shaded/pom.xml | 2 +- pulsar-proxy/pom.xml | 2 +- tiered-storage/jcloud/pom.xml | 2 +- 10 files changed, 14 insertions(+), 21 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index ae7ffaa7fc9e3..faea3fee824f0 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -389,8 +389,8 @@ The Apache Software License, Version 2.0 * AirCompressor - io.airlift-aircompressor-0.27.jar * AsyncHttpClient - - org.asynchttpclient-async-http-client-2.12.1.jar - - org.asynchttpclient-async-http-client-netty-utils-2.12.1.jar + - org.asynchttpclient-async-http-client-2.12.4.jar + - org.asynchttpclient-async-http-client-netty-utils-2.12.4.jar * Jetty - org.eclipse.jetty-jetty-client-9.4.56.v20240826.jar - org.eclipse.jetty-jetty-continuation-9.4.56.v20240826.jar @@ -570,7 +570,7 @@ Protocol Buffers License CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - - com.sun.activation-javax.activation-1.2.0.jar + - com.sun.activation-jakarta.activation-1.2.2.jar * Java Servlet API -- javax.servlet-javax.servlet-api-3.1.0.jar * WebSocket Server API -- javax.websocket-javax.websocket-client-api-1.0.jar * HK2 - Dependency Injection Kernel diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 0b3809431df1c..e4d3771185be3 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -399,8 +399,8 @@ The Apache Software License, Version 2.0 * AirCompressor - aircompressor-0.27.jar * AsyncHttpClient - - async-http-client-2.12.1.jar - - async-http-client-netty-utils-2.12.1.jar + - async-http-client-2.12.4.jar + - async-http-client-netty-utils-2.12.4.jar * Jetty - jetty-client-9.4.56.v20240826.jar - jetty-http-9.4.56.v20240826.jar @@ -431,7 +431,7 @@ MIT License CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - - javax.activation-1.2.0.jar + - jakarta.activation-1.2.2.jar * WebSocket Server API -- javax.websocket-client-api-1.0.jar * HK2 - Dependency Injection Kernel - hk2-api-2.6.1.jar diff --git a/pom.xml b/pom.xml index 34ba8e909d4ea..040c25a4a8b24 100644 --- a/pom.xml +++ b/pom.xml @@ -215,7 +215,7 @@ flexible messaging model and an intuitive client API. 0.16.1 6.2.8 0.27 - 2.12.1 + 2.12.4 3.11 1.10 2.18.0 @@ -233,7 +233,6 @@ flexible messaging model and an intuitive client API. 1.18.32 1.3.5 2.3.1 - 1.2.0 1.2.2 2.3.3 2.0.2 @@ -1441,12 +1440,6 @@ flexible messaging model and an intuitive client API. ${jakarta.xml.bind.version} - - com.sun.activation - javax.activation - ${javax.activation.version} - - com.sun.activation jakarta.activation diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index a34f566447006..7bd4eb89268ae 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -458,7 +458,7 @@ com.sun.activation - javax.activation + jakarta.activation diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 74ced063fbfd4..f667a8eb61e3f 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -111,7 +111,7 @@ com.google.re2j:re2j com.spotify:completable-futures com.squareup.*:* - com.sun.activation:javax.activation + com.sun.activation:jakarta.activation com.typesafe.netty:netty-reactive-streams com.yahoo.datasketches:* com.yahoo.datasketches:sketches-core diff --git a/pulsar-client-admin/pom.xml b/pulsar-client-admin/pom.xml index 0a94e48e9b939..12a1a7a6898dd 100644 --- a/pulsar-client-admin/pom.xml +++ b/pulsar-client-admin/pom.xml @@ -87,7 +87,7 @@ com.sun.activation - javax.activation + jakarta.activation runtime diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 74007745c70ee..5e30dbd999de4 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -152,7 +152,7 @@ com.google.re2j:re2j com.spotify:completable-futures com.squareup.*:* - com.sun.activation:javax.activation + com.sun.activation:jakarta.activation com.thoughtworks.paranamer:paranamer com.typesafe.netty:netty-reactive-streams diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index 1093b405731ea..62bab3cb2d71b 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -126,7 +126,7 @@ com.google.j2objc:* com.google.re2j:re2j com.spotify:completable-futures - com.sun.activation:javax.activation + com.sun.activation:jakarta.activation com.thoughtworks.paranamer:paranamer com.typesafe.netty:netty-reactive-streams diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 2cea6e0893005..6fca12d6660e9 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -140,7 +140,7 @@ com.sun.activation - javax.activation + jakarta.activation diff --git a/tiered-storage/jcloud/pom.xml b/tiered-storage/jcloud/pom.xml index 8fa504227feb1..dc28c3e05a0dc 100644 --- a/tiered-storage/jcloud/pom.xml +++ b/tiered-storage/jcloud/pom.xml @@ -117,7 +117,7 @@ com.sun.activation - javax.activation + jakarta.activation runtime From 069cc3db7c741a7b1fc64c79733ada847dca873e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Tue, 17 Dec 2024 11:38:13 +0800 Subject: [PATCH 201/327] [fix][admin] Verify is policies read only before revoke permissions on topic (#23730) --- .../admin/impl/PersistentTopicsBase.java | 3 +- .../broker/admin/PersistentTopicsTest.java | 30 +++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 9a306f6b4fff7..1300cd3449c27 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -289,7 +289,8 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges CompletableFuture validateAccessForTenantCf = - validateAdminAccessForTenantAsync(namespaceName.getTenant()); + validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()); var checkIfTopicExists = !pulsar().getConfiguration().isAllowAclChangesOnNonExistentTopics(); if (checkIfTopicExists) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index cca5049ed50eb..302948903442c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -1030,6 +1030,36 @@ public void testRevokePartitionedTopic() { } } + @Test + public void testRevokePartitionedTopicWithReadonlyPolicies() throws Exception { + final String partitionedTopicName = "testRevokePartitionedTopicWithReadonlyPolicies-topic"; + final int numPartitions = 5; + AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.createPartitionedTopic( + response, testTenant, testNamespace, partitionedTopicName, numPartitions, true); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); + String role = "role"; + Set expectActions = new HashSet<>(); + expectActions.add(AuthAction.produce); + response = mock(AsyncResponse.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.grantPermissionsOnTopic(response, testTenant, testNamespace, partitionedTopicName, role, + expectActions); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); + response = mock(AsyncResponse.class); + doReturn(CompletableFuture.failedFuture( + new RestException(Response.Status.FORBIDDEN, "Broker is forbidden to do read-write operations")) + ).when(persistentTopics).validatePoliciesReadOnlyAccessAsync(); + persistentTopics.revokePermissionsOnTopic(response, testTenant, testNamespace, partitionedTopicName, role); + ArgumentCaptor exceptionCaptor = ArgumentCaptor.forClass(RestException.class); + verify(response, timeout(5000).times(1)).resume(exceptionCaptor.capture()); + Assert.assertEquals(exceptionCaptor.getValue().getResponse().getStatus(), + Response.Status.FORBIDDEN.getStatusCode()); + } + @Test public void testTriggerCompactionTopic() { final String partitionTopicName = "test-part"; From 034791f66b3b91489ab2819f93c7f1fbcc3d8c38 Mon Sep 17 00:00:00 2001 From: Philipp Dolif <52791955+pdolif@users.noreply.github.com> Date: Tue, 17 Dec 2024 07:23:13 +0100 Subject: [PATCH 202/327] [fix][common] TopicName: Throw IllegalArgumentException if localName is whitespace only (#23691) --- .../java/org/apache/pulsar/common/naming/TopicName.java | 6 +++--- .../org/apache/pulsar/common/naming/TopicNameTest.java | 7 +++++++ 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java index dd24c9a971210..b2f96bfe6e259 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java @@ -170,9 +170,9 @@ private TopicName(String completeTopicName) { throw new IllegalArgumentException("Invalid topic name: " + completeTopicName); } - - if (localName == null || localName.isEmpty()) { - throw new IllegalArgumentException("Invalid topic name: " + completeTopicName); + if (StringUtils.isBlank(localName)) { + throw new IllegalArgumentException(String.format("Invalid topic name: %s. Topic local name must not" + + " be blank.", completeTopicName)); } } catch (NullPointerException e) { diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/TopicNameTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/TopicNameTest.java index 485bea3f1addb..27eb82d15af0d 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/TopicNameTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/TopicNameTest.java @@ -177,6 +177,13 @@ public void topic() { // Ok } + try { + TopicName.get(" "); + fail("Should have raised exception"); + } catch (IllegalArgumentException e) { + // Ok + } + TopicName nameWithSlash = TopicName.get("persistent://tenant/cluster/namespace/ns-abc/table/1"); assertEquals(nameWithSlash.getEncodedLocalName(), Codec.encode("ns-abc/table/1")); From 8d7d1fbb8e0d55bdd7a21f20f79247c22f20489d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 17 Dec 2024 12:39:25 +0200 Subject: [PATCH 203/327] [fix][sec] Upgrade golang.org/x/crypto from 0.21.0 to 0.31.0 in pulsar-function-go (#23743) --- pulsar-function-go/examples/go.mod | 8 ++++---- pulsar-function-go/examples/go.sum | 16 ++++++++-------- pulsar-function-go/go.mod | 8 ++++---- pulsar-function-go/go.sum | 16 ++++++++-------- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/pulsar-function-go/examples/go.mod b/pulsar-function-go/examples/go.mod index 0c2c6235b0fb6..449ea0988bdd8 100644 --- a/pulsar-function-go/examples/go.mod +++ b/pulsar-function-go/examples/go.mod @@ -42,12 +42,12 @@ require ( github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/stretchr/testify v1.8.4 // indirect go.uber.org/atomic v1.7.0 // indirect - golang.org/x/crypto v0.21.0 // indirect + golang.org/x/crypto v0.31.0 // indirect golang.org/x/net v0.23.0 // indirect golang.org/x/oauth2 v0.13.0 // indirect - golang.org/x/sys v0.18.0 // indirect - golang.org/x/term v0.18.0 // indirect - golang.org/x/text v0.14.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/term v0.27.0 // indirect + golang.org/x/text v0.21.0 // indirect google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect google.golang.org/grpc v1.60.0 // indirect diff --git a/pulsar-function-go/examples/go.sum b/pulsar-function-go/examples/go.sum index 37c84e71c8b26..e63985f12204f 100644 --- a/pulsar-function-go/examples/go.sum +++ b/pulsar-function-go/examples/go.sum @@ -393,8 +393,8 @@ golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= -golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= +golang.org/x/crypto v0.31.0 h1:ihbySMvVjLAeSH1IbfcRTkD/iNscyz8rGzjF/E5hV6U= +golang.org/x/crypto v0.31.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -559,12 +559,12 @@ golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= -golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= -golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= +golang.org/x/term v0.27.0 h1:WP60Sv1nlK1T6SupCHbXzSaN0b9wUmsPoRS9b61A23Q= +golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -575,8 +575,8 @@ golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= -golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= -golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= diff --git a/pulsar-function-go/go.mod b/pulsar-function-go/go.mod index 8dd3f4ef55473..3b4ac4f8a4b9e 100644 --- a/pulsar-function-go/go.mod +++ b/pulsar-function-go/go.mod @@ -45,12 +45,12 @@ require ( github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect go.uber.org/atomic v1.7.0 // indirect - golang.org/x/crypto v0.21.0 // indirect + golang.org/x/crypto v0.31.0 // indirect golang.org/x/net v0.23.0 // indirect golang.org/x/oauth2 v0.13.0 // indirect - golang.org/x/sys v0.18.0 // indirect - golang.org/x/term v0.18.0 // indirect - golang.org/x/text v0.14.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/term v0.27.0 // indirect + golang.org/x/text v0.21.0 // indirect google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/pulsar-function-go/go.sum b/pulsar-function-go/go.sum index 0acd26248a8fd..b0b3ce60fe59e 100644 --- a/pulsar-function-go/go.sum +++ b/pulsar-function-go/go.sum @@ -393,8 +393,8 @@ golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= -golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= +golang.org/x/crypto v0.31.0 h1:ihbySMvVjLAeSH1IbfcRTkD/iNscyz8rGzjF/E5hV6U= +golang.org/x/crypto v0.31.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -559,12 +559,12 @@ golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= -golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= -golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= +golang.org/x/term v0.27.0 h1:WP60Sv1nlK1T6SupCHbXzSaN0b9wUmsPoRS9b61A23Q= +golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -575,8 +575,8 @@ golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= -golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= -golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= From 8ad67776fc0787fe857d999a72f9df1e95e4210d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 17 Dec 2024 18:33:25 +0200 Subject: [PATCH 204/327] [improve][fn] Improve implementation for maxPendingAsyncRequests async concurrency limit when return type is CompletableFuture (#23708) --- .../instance/JavaExecutionResult.java | 4 +- .../functions/instance/JavaInstance.java | 77 +++++++++++++++---- .../instance/JavaInstanceRunnable.java | 2 +- .../functions/instance/JavaInstanceTest.java | 64 +++++++++++++++ 4 files changed, 127 insertions(+), 20 deletions(-) diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaExecutionResult.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaExecutionResult.java index 7af238154d634..5856600196b49 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaExecutionResult.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaExecutionResult.java @@ -27,13 +27,11 @@ */ @Data public class JavaExecutionResult { - private Exception userException; - private Exception systemException; + private Throwable userException; private Object result; public void reset() { setUserException(null); - setSystemException(null); setResult(null); } } diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java index 292f52b5091b9..5946be9fe5be9 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java @@ -25,11 +25,13 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; import java.util.function.Consumer; import lombok.AccessLevel; import lombok.Data; import lombok.Getter; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.functions.api.Function; import org.apache.pulsar.functions.api.Record; @@ -57,13 +59,26 @@ public static class AsyncFuncRequest { private final ExecutorService executor; @Getter private final LinkedBlockingQueue pendingAsyncRequests; + @Getter + private final Semaphore asyncRequestsConcurrencyLimiter; + private final boolean asyncPreserveInputOrderForOutputMessages; public JavaInstance(ContextImpl contextImpl, Object userClassObject, InstanceConfig instanceConfig) { this.context = contextImpl; this.instanceConfig = instanceConfig; this.executor = Executors.newSingleThreadExecutor(); - this.pendingAsyncRequests = new LinkedBlockingQueue<>(this.instanceConfig.getMaxPendingAsyncRequests()); + + asyncPreserveInputOrderForOutputMessages = + resolveAsyncPreserveInputOrderForOutputMessages(instanceConfig); + + if (asyncPreserveInputOrderForOutputMessages) { + this.pendingAsyncRequests = new LinkedBlockingQueue<>(this.instanceConfig.getMaxPendingAsyncRequests()); + this.asyncRequestsConcurrencyLimiter = null; + } else { + this.pendingAsyncRequests = null; + this.asyncRequestsConcurrencyLimiter = new Semaphore(this.instanceConfig.getMaxPendingAsyncRequests()); + } // create the functions if (userClassObject instanceof Function) { @@ -73,6 +88,20 @@ public JavaInstance(ContextImpl contextImpl, Object userClassObject, InstanceCon } } + // resolve whether to preserve input order for output messages for async functions + private boolean resolveAsyncPreserveInputOrderForOutputMessages(InstanceConfig instanceConfig) { + // no need to preserve input order for output messages if the function returns Void type + boolean voidReturnType = instanceConfig.getFunctionDetails() != null + && instanceConfig.getFunctionDetails().getSink() != null + && Void.class.getName().equals(instanceConfig.getFunctionDetails().getSink().getTypeClassName()); + if (voidReturnType) { + return false; + } + + // preserve input order for output messages + return true; + } + @VisibleForTesting public JavaExecutionResult handleMessage(Record record, Object input) { return handleMessage(record, input, (rec, result) -> { @@ -103,15 +132,33 @@ public JavaExecutionResult handleMessage(Record record, Object input, } if (output instanceof CompletableFuture) { - // Function is in format: Function> - AsyncFuncRequest request = new AsyncFuncRequest( - record, (CompletableFuture) output - ); try { - pendingAsyncRequests.put(request); - ((CompletableFuture) output).whenCompleteAsync((res, cause) -> { + if (asyncPreserveInputOrderForOutputMessages) { + // Function is in format: Function> + AsyncFuncRequest request = new AsyncFuncRequest( + record, (CompletableFuture) output + ); + pendingAsyncRequests.put(request); + } else { + asyncRequestsConcurrencyLimiter.acquire(); + } + ((CompletableFuture) output).whenCompleteAsync((Object res, Throwable cause) -> { try { - processAsyncResults(asyncResultConsumer); + if (asyncPreserveInputOrderForOutputMessages) { + processAsyncResultsInInputOrder(asyncResultConsumer); + } else { + try { + JavaExecutionResult execResult = new JavaExecutionResult(); + if (cause != null) { + execResult.setUserException(FutureUtil.unwrapCompletionException(cause)); + } else { + execResult.setResult(res); + } + asyncResultConsumer.accept(record, execResult); + } finally { + asyncRequestsConcurrencyLimiter.release(); + } + } } catch (Throwable innerException) { // the thread used for processing async results failed asyncFailureHandler.accept(innerException); @@ -132,21 +179,20 @@ public JavaExecutionResult handleMessage(Record record, Object input, } } - private void processAsyncResults(JavaInstanceRunnable.AsyncResultConsumer resultConsumer) throws Exception { + // processes the async results in the input order so that the order of the result messages in the output topic + // are in the same order as the input + private void processAsyncResultsInInputOrder(JavaInstanceRunnable.AsyncResultConsumer resultConsumer) + throws Exception { AsyncFuncRequest asyncResult = pendingAsyncRequests.peek(); while (asyncResult != null && asyncResult.getProcessResult().isDone()) { pendingAsyncRequests.remove(asyncResult); - JavaExecutionResult execResult = new JavaExecutionResult(); + JavaExecutionResult execResult = new JavaExecutionResult(); try { Object result = asyncResult.getProcessResult().get(); execResult.setResult(result); } catch (ExecutionException e) { - if (e.getCause() instanceof Exception) { - execResult.setUserException((Exception) e.getCause()); - } else { - execResult.setUserException(new Exception(e.getCause())); - } + execResult.setUserException(FutureUtil.unwrapCompletionException(e)); } resultConsumer.accept(asyncResult.getRecord(), execResult); @@ -154,7 +200,6 @@ private void processAsyncResults(JavaInstanceRunnable.AsyncResultConsumer result // peek the next result asyncResult = pendingAsyncRequests.peek(); } - } public void initialize() throws Exception { diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java index baf0c5f7400ec..4f811c14704a0 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java @@ -423,7 +423,7 @@ private StateStoreProvider getStateStoreProvider() throws Exception { @VisibleForTesting void handleResult(Record srcRecord, JavaExecutionResult result) throws Exception { if (result.getUserException() != null) { - Exception t = result.getUserException(); + Throwable t = result.getUserException(); log.warn("Encountered exception when processing message {}", srcRecord, t); stats.incrUserExceptions(t); diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceTest.java index 5a3332042938d..b3fcef292e52b 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceTest.java @@ -23,10 +23,13 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertTrue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.functions.api.Function; @@ -245,4 +248,65 @@ public UserException(String msg) { super(msg); } } + + @Test + public void testAsyncFunctionMaxPendingVoidResult() throws Exception { + CountDownLatch count = new CountDownLatch(1); + InstanceConfig instanceConfig = new InstanceConfig(); + instanceConfig.setFunctionDetails(org.apache.pulsar.functions.proto.Function.FunctionDetails.newBuilder() + .setSink(org.apache.pulsar.functions.proto.Function.SinkSpec.newBuilder() + .setTypeClassName(Void.class.getName()) + .build()) + .build()); + int pendingQueueSize = 3; + instanceConfig.setMaxPendingAsyncRequests(pendingQueueSize); + @Cleanup("shutdownNow") + ExecutorService executor = Executors.newCachedThreadPool(); + + Function> function = (input, context) -> { + CompletableFuture result = new CompletableFuture<>(); + executor.submit(() -> { + try { + count.await(); + result.complete(null); + } catch (Exception e) { + result.completeExceptionally(e); + } + }); + + return result; + }; + + JavaInstance instance = new JavaInstance( + mock(ContextImpl.class), + function, + instanceConfig); + String testString = "ABC123"; + + CountDownLatch resultsLatch = new CountDownLatch(3); + + long startTime = System.currentTimeMillis(); + assertEquals(pendingQueueSize, instance.getAsyncRequestsConcurrencyLimiter().availablePermits()); + JavaInstanceRunnable.AsyncResultConsumer asyncResultConsumer = (rec, result) -> { + resultsLatch.countDown(); + }; + Consumer asyncFailureHandler = cause -> { + }; + assertNull(instance.handleMessage(mock(Record.class), testString, asyncResultConsumer, asyncFailureHandler)); + assertEquals(pendingQueueSize - 1, instance.getAsyncRequestsConcurrencyLimiter().availablePermits()); + assertNull(instance.handleMessage(mock(Record.class), testString, asyncResultConsumer, asyncFailureHandler)); + assertEquals(pendingQueueSize - 2, instance.getAsyncRequestsConcurrencyLimiter().availablePermits()); + assertNull(instance.handleMessage(mock(Record.class), testString, asyncResultConsumer, asyncFailureHandler)); + // no space left + assertEquals(0, instance.getAsyncRequestsConcurrencyLimiter().availablePermits()); + + count.countDown(); + + assertTrue(resultsLatch.await(5, TimeUnit.SECONDS)); + + long endTime = System.currentTimeMillis(); + + log.info("start:{} end:{} during:{}", startTime, endTime, endTime - startTime); + instance.close(); + } } From 9f046a5f6bfa35bc89e74635cbf7aacd43bc9fc9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 17 Dec 2024 18:34:32 +0200 Subject: [PATCH 205/327] [improve][fn] Improve closing of producers in Pulsar Functions ProducerCache invalidation (#23734) --- .../functions/instance/ProducerCache.java | 38 ++++++++--- .../functions/instance/ProducerCacheTest.java | 64 +++++++++++++++++++ 2 files changed, 92 insertions(+), 10 deletions(-) create mode 100644 pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerCacheTest.java diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.java index f68c4e9589558..2e10581b35249 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.java @@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.util.FutureUtil; @Slf4j @@ -61,24 +62,41 @@ record ProducerCacheKey(CacheArea cacheArea, String topic, Object additionalKey) private final CopyOnWriteArrayList> closeFutures = new CopyOnWriteArrayList<>(); public ProducerCache() { - Caffeine builder = Caffeine.newBuilder() + Caffeine> builder = Caffeine.newBuilder() .scheduler(Scheduler.systemScheduler()) - .removalListener((key, producer, cause) -> { + .>removalListener((key, producer, cause) -> { log.info("Closing producer for topic {}, cause {}", key.topic(), cause); CompletableFuture closeFuture = - producer.flushAsync() + CompletableFuture.supplyAsync(() -> producer.flushAsync(), Runnable::run) .orTimeout(FLUSH_OR_CLOSE_TIMEOUT_SECONDS, TimeUnit.SECONDS) .exceptionally(ex -> { - log.error("Error flushing producer for topic {}", key.topic(), ex); + Throwable unwrappedCause = FutureUtil.unwrapCompletionException(ex); + if (unwrappedCause instanceof PulsarClientException.AlreadyClosedException) { + log.error( + "Error flushing producer for topic {} due to " + + "AlreadyClosedException", + key.topic()); + } else { + log.error("Error flushing producer for topic {}", key.topic(), + unwrappedCause); + } return null; }).thenCompose(__ -> producer.closeAsync().orTimeout(FLUSH_OR_CLOSE_TIMEOUT_SECONDS, - TimeUnit.SECONDS) - .exceptionally(ex -> { - log.error("Error closing producer for topic {}", key.topic(), - ex); - return null; - })); + TimeUnit.SECONDS) + ).exceptionally(ex -> { + Throwable unwrappedCause = FutureUtil.unwrapCompletionException(ex); + if (unwrappedCause instanceof PulsarClientException.AlreadyClosedException) { + log.error( + "Error closing producer for topic {} due to " + + "AlreadyClosedException", + key.topic()); + } else { + log.error("Error closing producer for topic {}", key.topic(), + unwrappedCause); + } + return null; + }); if (closed.get()) { closeFutures.add(closeFuture); } diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerCacheTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerCacheTest.java new file mode 100644 index 0000000000000..af95a7901b6e8 --- /dev/null +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerCacheTest.java @@ -0,0 +1,64 @@ +/* + * 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.functions.instance; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import java.util.concurrent.CompletableFuture; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; +import org.testng.annotations.Test; + +public class ProducerCacheTest { + + @Test + public void shouldTolerateAlreadyClosedExceptionInClose() { + ProducerCache cache = new ProducerCache(); + Producer producer = mock(Producer.class); + when(producer.flushAsync()).thenReturn(CompletableFuture.completedFuture(null)); + when(producer.closeAsync()).thenReturn( + CompletableFuture.failedFuture(new PulsarClientException.AlreadyClosedException("Already closed"))); + cache.getOrCreateProducer(ProducerCache.CacheArea.CONTEXT_CACHE, "topic", "key", + () -> (Producer) producer); + cache.close(); + } + + @Test + public void shouldTolerateRuntimeExceptionInClose() { + ProducerCache cache = new ProducerCache(); + Producer producer = mock(Producer.class); + when(producer.flushAsync()).thenReturn(CompletableFuture.completedFuture(null)); + when(producer.closeAsync()).thenThrow(new RuntimeException("Some exception")); + cache.getOrCreateProducer(ProducerCache.CacheArea.CONTEXT_CACHE, "topic", "key", + () -> (Producer) producer); + cache.close(); + } + + @Test + public void shouldTolerateRuntimeExceptionInFlush() { + ProducerCache cache = new ProducerCache(); + Producer producer = mock(Producer.class); + when(producer.flushAsync()).thenThrow(new RuntimeException("Some exception")); + when(producer.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); + cache.getOrCreateProducer(ProducerCache.CacheArea.CONTEXT_CACHE, "topic", "key", + () -> (Producer) producer); + cache.close(); + } + +} \ No newline at end of file From 1c9bf82b7c88af9175faf76112c76d06db64df75 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 18 Dec 2024 17:17:24 +0800 Subject: [PATCH 206/327] [improve] Upgrade lombok to 1.18.36 (#23752) Signed-off-by: Zixuan Liu --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 040c25a4a8b24..bd7320ef1c32d 100644 --- a/pom.xml +++ b/pom.xml @@ -230,7 +230,7 @@ flexible messaging model and an intuitive client API. 0.9.1 2.1.0 3.24.2 - 1.18.32 + 1.18.36 1.3.5 2.3.1 1.2.2 From e535d990f60b5c15f1ec440e82de2fa80d6783a5 Mon Sep 17 00:00:00 2001 From: JHC Date: Thu, 19 Dec 2024 16:23:34 +0800 Subject: [PATCH 207/327] [fix][broker] Fix typo in `buildMetadataForCompactedLedger` (#23755) --- .../org/apache/bookkeeper/mledger/impl/LedgerMetadataUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerMetadataUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerMetadataUtils.java index 4ac409a2e9bfe..4107949de51b9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerMetadataUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerMetadataUtils.java @@ -80,7 +80,7 @@ static Map buildAdditionalMetadataForCursor(String name) { * Build additional metadata for a CompactedLedger. * * @param compactedTopic reference to the compacted topic. - * @param compactedToMessageId last mesasgeId. + * @param compactedToMessageId last messageId. * @return an immutable map which describes the compacted ledger */ public static Map buildMetadataForCompactedLedger(String compactedTopic, From ab1b5c00565adfe877719130127fc23ac9c5a0c1 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 20 Dec 2024 10:59:04 +0800 Subject: [PATCH 208/327] [fix] [client] Fix memory leak when publishing encountered a corner case error (#23738) Co-authored-by: Yunze Xu --- .../client/impl/ProducerMemoryLeakTest.java | 364 ++++++++++++++++++ .../impl/BatchMessageContainerImpl.java | 4 +- .../pulsar/client/impl/ProducerImpl.java | 54 ++- .../client/impl/ProducerInterceptors.java | 12 +- 4 files changed, 417 insertions(+), 17 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLeakTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLeakTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLeakTest.java new file mode 100644 index 0000000000000..dcdfd136476c3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLeakTest.java @@ -0,0 +1,364 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.impl; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mockStatic; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import io.netty.buffer.ByteBuf; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.interceptor.ProducerInterceptor; +import org.apache.pulsar.common.protocol.ByteBufPair; +import org.apache.pulsar.common.util.FutureUtil; +import org.awaitility.Awaitility; +import org.mockito.MockedStatic; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class ProducerMemoryLeakTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testSendQueueIsFull() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp_"); + admin.topics().createNonPartitionedTopic(topicName); + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer(Schema.STRING) + .blockIfQueueFull(false).maxPendingMessages(1) + .enableBatching(true).topic(topicName).create(); + List> msgBuilderList = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + msgBuilderList.add(newMessage(producer)); + } + + CompletableFuture latestSendFuture = null; + for (MsgPayloadTouchableMessageBuilder msgBuilder: msgBuilderList) { + latestSendFuture = msgBuilder.value("msg-1").sendAsync(); + } + try{ + latestSendFuture.join(); + } catch (Exception ex) { + // Ignore the error PulsarClientException$ProducerQueueIsFullError. + assertTrue(FutureUtil.unwrapCompletionException(ex) + instanceof PulsarClientException.ProducerQueueIsFullError); + } + + // Verify: ref is expected. + producer.close(); + for (int i = 0; i < msgBuilderList.size(); i++) { + MsgPayloadTouchableMessageBuilder msgBuilder = msgBuilderList.get(i); + assertEquals(msgBuilder.payload.refCnt(), 1); + msgBuilder.release(); + assertEquals(msgBuilder.payload.refCnt(), 0); + } + admin.topics().delete(topicName); + } + + /** + * The content size of msg(value is "msg-1") will be "5". + * Then provides two param: 1 and 5. + * 1: reach the limitation before adding the message metadata. + * 2: reach the limitation after adding the message metadata. + */ + @DataProvider(name = "maxMessageSizeAndCompressions") + public Object[][] maxMessageSizeAndCompressions(){ + return new Object[][] { + {1, CompressionType.NONE}, + {5, CompressionType.NONE}, + {1, CompressionType.LZ4}, + {6, CompressionType.LZ4} + }; + } + + @Test(dataProvider = "maxMessageSizeAndCompressions") + public void testSendMessageSizeExceeded(int maxMessageSize, CompressionType compressionType) throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp_"); + admin.topics().createNonPartitionedTopic(topicName); + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer(Schema.STRING).topic(topicName) + .compressionType(compressionType) + .enableBatching(false) + .create(); + producer.getConnectionHandler().setMaxMessageSize(maxMessageSize); + MsgPayloadTouchableMessageBuilder msgBuilder = newMessage(producer); + /** + * Mock an error: reached max message size, see more details {@link #maxMessageSizeAndCompressions()}. + */ + try (MockedStatic theMock = mockStatic(ByteBufPair.class)) { + List generatedByteBufPairs = Collections.synchronizedList(new ArrayList<>()); + theMock.when(() -> ByteBufPair.get(any(ByteBuf.class), any(ByteBuf.class))).then(invocation -> { + ByteBufPair byteBufPair = (ByteBufPair) invocation.callRealMethod(); + generatedByteBufPairs.add(byteBufPair); + byteBufPair.retain(); + return byteBufPair; + }); + try { + msgBuilder.value("msg-1").send(); + fail("expected an error that reached the max message size"); + } catch (Exception ex) { + assertTrue(FutureUtil.unwrapCompletionException(ex) + instanceof PulsarClientException.InvalidMessageException); + } + + // Verify: message payload has been released. + // Since "MsgPayloadTouchableMessageBuilder" has called "buffer.retain" once, "refCnt()" should be "1". + producer.close(); + Awaitility.await().untilAsserted(() -> { + assertEquals(producer.getPendingQueueSize(), 0); + }); + // Verify: ByteBufPair generated for Pulsar Command. + if (maxMessageSize == 1) { + assertEquals(generatedByteBufPairs.size(),0); + } else { + assertEquals(generatedByteBufPairs.size(),1); + if (compressionType == CompressionType.NONE) { + assertEquals(msgBuilder.payload.refCnt(), 2); + } else { + assertEquals(msgBuilder.payload.refCnt(), 1); + } + for (ByteBufPair byteBufPair : generatedByteBufPairs) { + assertEquals(byteBufPair.refCnt(), 1); + byteBufPair.release(); + assertEquals(byteBufPair.refCnt(), 0); + } + } + // Verify: message.payload + assertEquals(msgBuilder.payload.refCnt(), 1); + msgBuilder.release(); + assertEquals(msgBuilder.payload.refCnt(), 0); + } + + // cleanup. + assertEquals(msgBuilder.payload.refCnt(), 0); + admin.topics().delete(topicName); + } + + /** + * The content size of msg(value is "msg-1") will be "5". + * Then provides two param: 1 and 5. + * 1: Less than the limitation when adding the message into the batch-container. + * 3: Less than the limitation when building batched messages payload. + * 2: Equals the limitation when building batched messages payload. + */ + @DataProvider(name = "maxMessageSizes") + public Object[][] maxMessageSizes(){ + return new Object[][] { + {1}, + {3}, + {26} + }; + } + + @Test(dataProvider = "maxMessageSizes") + public void testBatchedSendMessageSizeExceeded(int maxMessageSize) throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp_"); + admin.topics().createNonPartitionedTopic(topicName); + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer(Schema.STRING).topic(topicName) + .enableBatching(true) + .compressionType(CompressionType.NONE) + .create(); + final ClientCnx cnx = producer.getClientCnx(); + producer.getConnectionHandler().setMaxMessageSize(maxMessageSize); + MsgPayloadTouchableMessageBuilder msgBuilder1 = newMessage(producer); + MsgPayloadTouchableMessageBuilder msgBuilder2 = newMessage(producer); + /** + * Mock an error: reached max message size. see more detail {@link #maxMessageSizes()}. + */ + msgBuilder1.value("msg-1").sendAsync(); + try { + msgBuilder2.value("msg-1").send(); + if (maxMessageSize != 26) { + fail("expected an error that reached the max message size"); + } + } catch (Exception ex) { + assertTrue(FutureUtil.unwrapCompletionException(ex) + instanceof PulsarClientException.InvalidMessageException); + } + + // Verify: message payload has been released. + // Since "MsgPayloadTouchableMessageBuilder" has called "buffer.retain" once, "refCnt()" should be "1". + producer.close(); + Awaitility.await().untilAsserted(() -> { + assertEquals(producer.getPendingQueueSize(), 0); + }); + assertEquals(msgBuilder1.payload.refCnt(), 1); + assertEquals(msgBuilder2.payload.refCnt(), 1); + + // cleanup. + cnx.ctx().close(); + msgBuilder1.release(); + msgBuilder2.release(); + assertEquals(msgBuilder1.payload.refCnt(), 0); + assertEquals(msgBuilder2.payload.refCnt(), 0); + admin.topics().delete(topicName); + } + + @Test + public void testSendAfterClosedProducer() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp_"); + admin.topics().createNonPartitionedTopic(topicName); + ProducerImpl producer = + (ProducerImpl) pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + // Publish after the producer was closed. + MsgPayloadTouchableMessageBuilder msgBuilder = newMessage(producer); + producer.close(); + try { + msgBuilder.value("msg-1").send(); + fail("expected an error that the producer has closed"); + } catch (Exception ex) { + assertTrue(FutureUtil.unwrapCompletionException(ex) + instanceof PulsarClientException.AlreadyClosedException); + } + + // Verify: message payload has been released. + Awaitility.await().untilAsserted(() -> { + assertEquals(producer.getPendingQueueSize(), 0); + }); + assertEquals(msgBuilder.payload.refCnt(), 1); + + // cleanup. + msgBuilder.release(); + assertEquals(msgBuilder.payload.refCnt(), 0); + admin.topics().delete(topicName); + } + + @DataProvider + public Object[][] failedInterceptAt() { + return new Object[][]{ + {"close"}, + {"eligible"}, + {"beforeSend"}, + {"onSendAcknowledgement"}, + }; + } + + @Test(dataProvider = "failedInterceptAt") + public void testInterceptorError(String method) throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp_"); + admin.topics().createNonPartitionedTopic(topicName); + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer(Schema.STRING).topic(topicName) + .intercept( + + new ProducerInterceptor() { + @Override + public void close() { + if (method.equals("close")) { + throw new RuntimeException("Mocked error"); + } + } + + @Override + public boolean eligible(Message message) { + if (method.equals("eligible")) { + throw new RuntimeException("Mocked error"); + } + return false; + } + + @Override + public Message beforeSend(Producer producer, Message message) { + if (method.equals("beforeSend")) { + throw new RuntimeException("Mocked error"); + } + return message; + } + + @Override + public void onSendAcknowledgement(Producer producer, Message message, MessageId msgId, + Throwable exception) { + if (method.equals("onSendAcknowledgement")) { + throw new RuntimeException("Mocked error"); + } + + } + }).create(); + + MsgPayloadTouchableMessageBuilder msgBuilder = newMessage(producer); + try { + msgBuilder.value("msg-1").sendAsync().get(3, TimeUnit.SECONDS); + // It may throw error. + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("Mocked")); + } + + // Verify: message payload has been released. + producer.close(); + assertEquals(msgBuilder.payload.refCnt(), 1); + + // cleanup. + msgBuilder.release(); + assertEquals(msgBuilder.payload.refCnt(), 0); + admin.topics().delete(topicName); + } + + private MsgPayloadTouchableMessageBuilder newMessage(ProducerImpl producer){ + return new MsgPayloadTouchableMessageBuilder(producer, producer.schema); + } + + private static class MsgPayloadTouchableMessageBuilder extends TypedMessageBuilderImpl { + + public volatile ByteBuf payload; + + public MsgPayloadTouchableMessageBuilder(ProducerBase producer, Schema schema) { + super(producer, schema); + } + + @Override + public Message getMessage() { + MessageImpl msg = (MessageImpl) super.getMessage(); + payload = msg.getPayload(); + // Retain the msg to avoid it be reused by other task. + payload.retain(); + return msg; + } + + public void release() { + payload.release(); + } + } +} 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 44f1fb274655a..7262cfd11e069 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 @@ -290,8 +290,8 @@ public OpSendMsg createOpSendMsg() throws IOException { messages.forEach(msg -> producer.client.getMemoryLimitController() .releaseMemory(msg.getUncompressedSize())); producer.client.getMemoryLimitController().releaseMemory(batchAllocatedSizeBytes); - discard(new PulsarClientException.InvalidMessageException( - "Message size is bigger than " + getMaxMessageSize() + " bytes")); + discard(new PulsarClientException.InvalidMessageException("Message size " + + encryptedPayload.readableBytes() + " is bigger than " + getMaxMessageSize() + " bytes")); return null; } messageMetadata.setNumMessagesInBatch(numMessagesInBatch); 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 b686252b58ade..10e0ee2ee3d9f 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 @@ -32,6 +32,9 @@ import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandler; +import io.netty.channel.ChannelPromise; import io.netty.util.AbstractReferenceCounted; import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; @@ -483,19 +486,46 @@ private ByteBuf applyCompression(ByteBuf payload) { return compressedPayload; } + /** + * Note on ByteBuf Release Behavior. + * + *

If you have a customized callback, please ignore the note below.

+ * + *

When using the default callback, please confirm that the {@code refCnt()} value of the {@code message} + * (as returned by {@link MessageImpl#getDataBuffer}) is {@code 2} when you call this method. This is because + * the {@code ByteBuf} will be released twice under the following conditions:

+ * + *
    + *
  • Batch Messaging Enabled: + *
      + *
    1. Release 1: When the message is pushed into the batched message queue (see {@link #doBatchSendAndAdd}). + *
    2. + *
    3. Release 2: In the method {@link SendCallback#sendComplete(Throwable, OpSendMsgStats)}.
    4. + *
    + *
  • + *
  • Single Message (Batch Messaging Disabled): + *
      + *
    1. Release 1: When the message is written out by + * {@link ChannelOutboundHandler#write(ChannelHandlerContext, Object, ChannelPromise)}.
    2. + *
    3. Release 2: In the method {@link SendCallback#sendComplete(Throwable, OpSendMsgStats)}.
    4. + *
    + *
  • + *
+ */ public void sendAsync(Message message, SendCallback callback) { checkArgument(message instanceof MessageImpl); - - if (!isValidProducerState(callback, message.getSequenceId())) { - return; - } - MessageImpl msg = (MessageImpl) message; MessageMetadata msgMetadata = msg.getMessageBuilder(); ByteBuf payload = msg.getDataBuffer(); final int uncompressedSize = payload.readableBytes(); + if (!isValidProducerState(callback, message.getSequenceId())) { + payload.release(); + return; + } + if (!canEnqueueRequest(callback, message.getSequenceId(), uncompressedSize)) { + payload.release(); return; } @@ -573,6 +603,7 @@ public void sendAsync(Message message, SendCallback callback) { for (int i = 0; i < (totalChunks - 1); i++) { if (!conf.isBlockIfQueueFull() && !canEnqueueRequest(callback, message.getSequenceId(), 0 /* The memory was already reserved */)) { + compressedPayload.release(); client.getMemoryLimitController().releaseMemory(uncompressedSize); semaphoreRelease(i + 1); return; @@ -603,6 +634,7 @@ public void sendAsync(Message message, SendCallback callback) { } if (chunkId > 0 && conf.isBlockIfQueueFull() && !canEnqueueRequest(callback, message.getSequenceId(), 0 /* The memory was already reserved */)) { + compressedPayload.release(); client.getMemoryLimitController().releaseMemory(uncompressedSize - readStartIndex); semaphoreRelease(totalChunks - chunkId); return; @@ -723,10 +755,13 @@ private void serializeAndSendMessage(MessageImpl msg, } else { // handle boundary cases where message being added would exceed // batch size and/or max message size - boolean isBatchFull = batchMessageContainer.add(msg, callback); - lastSendFuture = callback.getFuture(); - payload.release(); - triggerSendIfFullOrScheduleFlush(isBatchFull); + try { + boolean isBatchFull = batchMessageContainer.add(msg, callback); + lastSendFuture = callback.getFuture(); + triggerSendIfFullOrScheduleFlush(isBatchFull); + } finally { + payload.release(); + } } isLastSequenceIdPotentialDuplicated = false; } @@ -2304,6 +2339,7 @@ protected void processOpSendMsg(OpSendMsg op) { batchMessageAndSend(false); } if (isMessageSizeExceeded(op)) { + op.cmd.release(); return; } pendingMessages.add(op); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerInterceptors.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerInterceptors.java index 97f16c37b5d31..38492ceae849b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerInterceptors.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerInterceptors.java @@ -60,10 +60,10 @@ public ProducerInterceptors(List interceptors) { public Message beforeSend(Producer producer, Message message) { Message interceptorMessage = message; for (ProducerInterceptor interceptor : interceptors) { - if (!interceptor.eligible(message)) { - continue; - } try { + if (!interceptor.eligible(message)) { + continue; + } interceptorMessage = interceptor.beforeSend(producer, interceptorMessage); } catch (Throwable e) { if (producer != null) { @@ -93,10 +93,10 @@ public Message beforeSend(Producer producer, Message message) { */ public void onSendAcknowledgement(Producer producer, Message message, MessageId msgId, Throwable exception) { for (ProducerInterceptor interceptor : interceptors) { - if (!interceptor.eligible(message)) { - continue; - } try { + if (!interceptor.eligible(message)) { + continue; + } interceptor.onSendAcknowledgement(producer, message, msgId, exception); } catch (Throwable e) { log.warn("Error executing interceptor onSendAcknowledgement callback ", e); From db892dd1f5f8c75155bd1bd7193a6144002cc77a Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Fri, 20 Dec 2024 17:30:38 +0800 Subject: [PATCH 209/327] [improve][monitor] Upgrade OTel to 1.45.0 (#23756) --- .../server/src/assemble/LICENSE.bin.txt | 43 ++++++++++--------- .../shell/src/assemble/LICENSE.bin.txt | 6 +-- pom.xml | 4 +- 3 files changed, 27 insertions(+), 26 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index faea3fee824f0..2182142cd6428 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -338,11 +338,12 @@ The Apache Software License, Version 2.0 - io.prometheus-simpleclient_tracer_otel-0.16.0.jar - io.prometheus-simpleclient_tracer_otel_agent-0.16.0.jar * Prometheus exporter - - io.prometheus-prometheus-metrics-config-1.3.3.jar - - io.prometheus-prometheus-metrics-exporter-common-1.3.3.jar - - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.3.jar - - io.prometheus-prometheus-metrics-exposition-formats-1.3.3.jar - - io.prometheus-prometheus-metrics-model-1.3.3.jar + - io.prometheus-prometheus-metrics-config-1.3.4.jar + - io.prometheus-prometheus-metrics-exporter-common-1.3.4.jar + - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.4.jar + - io.prometheus-prometheus-metrics-exposition-formats-1.3.4.jar + - io.prometheus-prometheus-metrics-model-1.3.4.jar + - io.prometheus-prometheus-metrics-exposition-textformats-1.3.4.jar * Jakarta Bean Validation API - jakarta.validation-jakarta.validation-api-2.0.2.jar - javax.validation-validation-api-1.1.0.Final.jar @@ -515,27 +516,27 @@ The Apache Software License, Version 2.0 * RoaringBitmap - org.roaringbitmap-RoaringBitmap-1.2.0.jar * OpenTelemetry - - io.opentelemetry-opentelemetry-api-1.44.1.jar - - io.opentelemetry-opentelemetry-api-incubator-1.44.1-alpha.jar - - io.opentelemetry-opentelemetry-context-1.44.1.jar - - io.opentelemetry-opentelemetry-exporter-common-1.44.1.jar - - io.opentelemetry-opentelemetry-exporter-otlp-1.44.1.jar - - io.opentelemetry-opentelemetry-exporter-otlp-common-1.44.1.jar - - io.opentelemetry-opentelemetry-exporter-prometheus-1.44.1-alpha.jar - - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.44.1.jar - - io.opentelemetry-opentelemetry-sdk-1.44.1.jar - - io.opentelemetry-opentelemetry-sdk-common-1.44.1.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.44.1.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.44.1.jar - - io.opentelemetry-opentelemetry-sdk-logs-1.44.1.jar - - io.opentelemetry-opentelemetry-sdk-metrics-1.44.1.jar - - io.opentelemetry-opentelemetry-sdk-trace-1.44.1.jar + - io.opentelemetry-opentelemetry-api-1.45.0.jar + - io.opentelemetry-opentelemetry-api-incubator-1.45.0-alpha.jar + - io.opentelemetry-opentelemetry-context-1.45.0.jar + - io.opentelemetry-opentelemetry-exporter-common-1.45.0.jar + - io.opentelemetry-opentelemetry-exporter-otlp-1.45.0.jar + - io.opentelemetry-opentelemetry-exporter-otlp-common-1.45.0.jar + - io.opentelemetry-opentelemetry-exporter-prometheus-1.45.0-alpha.jar + - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.45.0.jar + - io.opentelemetry-opentelemetry-sdk-1.45.0.jar + - io.opentelemetry-opentelemetry-sdk-common-1.45.0.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.45.0.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.45.0.jar + - io.opentelemetry-opentelemetry-sdk-logs-1.45.0.jar + - io.opentelemetry-opentelemetry-sdk-metrics-1.45.0.jar + - io.opentelemetry-opentelemetry-sdk-trace-1.45.0.jar - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.33.6.jar - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.33.6-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-resources-1.33.6-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.6-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.6-alpha.jar - - io.opentelemetry.semconv-opentelemetry-semconv-1.28.0-alpha.jar + - io.opentelemetry.semconv-opentelemetry-semconv-1.29.0-alpha.jar * Spotify completable-futures - com.spotify-completable-futures-0.3.6.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index e4d3771185be3..05342d1724399 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -388,9 +388,9 @@ The Apache Software License, Version 2.0 - log4j-slf4j2-impl-2.23.1.jar - log4j-web-2.23.1.jar * OpenTelemetry - - opentelemetry-api-1.44.1.jar - - opentelemetry-api-incubator-1.44.1-alpha.jar - - opentelemetry-context-1.44.1.jar + - opentelemetry-api-1.45.0.jar + - opentelemetry-api-incubator-1.45.0-alpha.jar + - opentelemetry-context-1.45.0.jar * BookKeeper - bookkeeper-common-allocator-4.17.1.jar diff --git a/pom.xml b/pom.xml index bd7320ef1c32d..2a91978af8e58 100644 --- a/pom.xml +++ b/pom.xml @@ -257,11 +257,11 @@ flexible messaging model and an intuitive client API. 3.4.3 1.5.2-3 2.0.6 - 1.44.1 + 1.45.0 ${opentelemetry.version}-alpha 1.33.6 ${opentelemetry.instrumentation.version}-alpha - 1.28.0-alpha + 1.29.0-alpha 4.7.5 1.7 0.3.6 From e0a9e4c7b5d3533f2a1e5b7757b180168412c35e Mon Sep 17 00:00:00 2001 From: pengxiangrui127 <67997328+pengxiangrui127@users.noreply.github.com> Date: Fri, 20 Dec 2024 19:17:13 +0800 Subject: [PATCH 210/327] [Fix][Client] Fix pending message not complete when closeAsync (#23761) --- .../pulsar/client/impl/ProducerImpl.java | 5 ++- .../pulsar/client/impl/ProducerImplTest.java | 43 +++++++++++++++++-- 2 files changed, 43 insertions(+), 5 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 10e0ee2ee3d9f..54d337925dcf9 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 @@ -1177,11 +1177,11 @@ public CompletableFuture closeAsync() { CompletableFuture closeFuture = new CompletableFuture<>(); cnx.sendRequestWithId(cmd, requestId).handle((v, exception) -> { cnx.removeProducer(producerId); + closeAndClearPendingMessages(); if (exception == null || !cnx.ctx().channel().isActive()) { // Either we've received the success response for the close producer command from the broker, or the // connection did break in the meantime. In any case, the producer is gone. log.info("[{}] [{}] Closed Producer", topic, producerName); - closeAndClearPendingMessages(); closeFuture.complete(null); } else { closeFuture.completeExceptionally(exception); @@ -1193,7 +1193,8 @@ public CompletableFuture closeAsync() { return closeFuture; } - private synchronized void closeAndClearPendingMessages() { + @VisibleForTesting + protected synchronized void closeAndClearPendingMessages() { setState(State.Closed); client.cleanupProducer(this); PulsarClientException ex = new PulsarClientException.AlreadyClosedException( diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java index f9df63759394a..5f690ead6c592 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java @@ -22,12 +22,19 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.mockito.Mockito.withSettings; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertTrue; +import static org.testng.Assert.*; + import java.nio.ByteBuffer; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +import io.netty.util.HashedWheelTimer; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.client.impl.metrics.LatencyHistogram; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.mockito.Mockito; import org.testng.annotations.Test; @@ -68,4 +75,34 @@ public void testPopulateMessageSchema() { verify(msg).setSchemaState(MessageImpl.SchemaState.Ready); } + @Test + public void testClearPendingMessageWhenCloseAsync() { + PulsarClientImpl client = mock(PulsarClientImpl.class); + Mockito.doReturn(1L).when(client).newProducerId(); + ClientConfigurationData clientConf = new ClientConfigurationData(); + clientConf.setStatsIntervalSeconds(-1); + Mockito.doReturn(clientConf).when(client).getConfiguration(); + Mockito.doReturn(new InstrumentProvider(null)).when(client).instrumentProvider(); + ConnectionPool connectionPool = mock(ConnectionPool.class); + Mockito.doReturn(1).when(connectionPool).genRandomKeyToSelectCon(); + Mockito.doReturn(connectionPool).when(client).getCnxPool(); + HashedWheelTimer timer = mock(HashedWheelTimer.class); + Mockito.doReturn(null).when(timer).newTimeout(Mockito.any(), Mockito.anyLong(), Mockito.any()); + Mockito.doReturn(timer).when(client).timer(); + ProducerConfigurationData producerConf = new ProducerConfigurationData(); + producerConf.setSendTimeoutMs(-1); + ProducerImpl producer = Mockito.spy(new ProducerImpl<>(client, "topicName", producerConf, null, 0, null, null, Optional.empty())); + + // make sure throw exception when send request to broker + ClientCnx clientCnx = mock(ClientCnx.class); + CompletableFuture tCompletableFuture = new CompletableFuture<>(); + tCompletableFuture.completeExceptionally(new PulsarClientException("error")); + when(clientCnx.sendRequestWithId(Mockito.any(), Mockito.anyLong())).thenReturn(tCompletableFuture); + Mockito.doReturn(clientCnx).when(producer).cnx(); + + // run closeAsync and verify + CompletableFuture voidCompletableFuture = producer.closeAsync(); + verify(producer).closeAndClearPendingMessages(); + } + } From a54be73e95e02f12e6656acb46c339c81dc0e531 Mon Sep 17 00:00:00 2001 From: Philipp Dolif <52791955+pdolif@users.noreply.github.com> Date: Fri, 20 Dec 2024 12:39:05 +0100 Subject: [PATCH 211/327] [fix][test] Fix flaky KeySharedSubscriptionTest.testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector (#23747) --- .../apache/pulsar/client/api/KeySharedSubscriptionTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 08efb6d9583ef..92257c1df53f8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -391,13 +391,14 @@ public void testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelect @Cleanup Producer producer = createProducer(topic, enableBatch); - for (int i = 0; i < 100; i++) { + int totalMessages = 300; + for (int i = 0; i < totalMessages; i++) { producer.newMessage() .value(i) .send(); } - receiveAndCheckDistribution(Lists.newArrayList(consumer1, consumer2, consumer3), 100); + receiveAndCheckDistribution(Lists.newArrayList(consumer1, consumer2, consumer3), totalMessages); } @Test(dataProvider = "batch") From bbe2cabc3ec0375607cb12665cab0b4745dbd36e Mon Sep 17 00:00:00 2001 From: hrzzzz <64506104+hrzzzz@users.noreply.github.com> Date: Fri, 20 Dec 2024 20:04:24 +0800 Subject: [PATCH 212/327] [fix][broker] Fix bug causing loss of migrated information when setting other localPolicies in namespace (#23764) Co-authored-by: ruihongzhou --- .../broker/admin/impl/NamespacesBase.java | 20 +++++++++----- .../common/naming/NamespaceBundles.java | 3 ++- .../pulsar/broker/admin/NamespacesTest.java | 27 +++++++++++++++++++ .../common/policies/data/LocalPolicies.java | 11 +++++++- 4 files changed, 52 insertions(+), 9 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index d80e2487b4f1c..ca4c685b2806a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -960,7 +960,8 @@ protected void internalSetBookieAffinityGroup(BookieAffinityGroupData bookieAffi LocalPolicies localPolicies = oldPolicies.map( policies -> new LocalPolicies(policies.bundles, bookieAffinityGroup, - policies.namespaceAntiAffinityGroup)) + policies.namespaceAntiAffinityGroup, + policies.migrated)) .orElseGet(() -> new LocalPolicies(getBundles(config().getDefaultNumberOfNamespaceBundles()), bookieAffinityGroup, null)); @@ -1779,7 +1780,8 @@ protected void internalSetNamespaceAntiAffinityGroup(String antiAffinityGroup) { getLocalPolicies().setLocalPoliciesWithCreate(namespaceName, (lp)-> lp.map(policies -> new LocalPolicies(policies.bundles, policies.bookieAffinityGroup, - antiAffinityGroup)) + antiAffinityGroup, + policies.migrated)) .orElseGet(() -> new LocalPolicies(defaultBundle(), null, antiAffinityGroup)) ); @@ -1816,7 +1818,8 @@ protected void internalRemoveNamespaceAntiAffinityGroup() { getLocalPolicies().setLocalPolicies(namespaceName, (policies)-> new LocalPolicies(policies.bundles, policies.bookieAffinityGroup, - null)); + null, + policies.migrated)); log.info("[{}] Successfully removed anti-affinity group for a namespace={}", clientAppId(), namespaceName); } catch (Exception e) { log.error("[{}] Failed to remove anti-affinity group for namespace {}", clientAppId(), namespaceName, e); @@ -2765,10 +2768,13 @@ protected void internalRemoveBacklogQuota(AsyncResponse asyncResponse, BacklogQu protected void internalEnableMigration(boolean migrated) { validateSuperUserAccess(); try { - getLocalPolicies().setLocalPolicies(namespaceName, (policies) -> { - policies.migrated = migrated; - return policies; - }); + getLocalPolicies().setLocalPoliciesWithCreate(namespaceName, oldPolicies -> oldPolicies.map( + policies -> new LocalPolicies(policies.bundles, + policies.bookieAffinityGroup, + policies.namespaceAntiAffinityGroup, + migrated)) + .orElseGet(() -> new LocalPolicies(getBundles(config().getDefaultNumberOfNamespaceBundles()), + null, null, migrated))); log.info("Successfully updated migration on namespace {}", namespaceName); } catch (Exception e) { log.error("Failed to update migration on namespace {}", namespaceName, e); 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..3ee365cdd4571 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 @@ -197,6 +197,7 @@ public BundlesData getBundlesData() { public LocalPolicies toLocalPolicies() { return new LocalPolicies(this.getBundlesData(), localPolicies.map(lp -> lp.getLeft().bookieAffinityGroup).orElse(null), - localPolicies.map(lp -> lp.getLeft().namespaceAntiAffinityGroup).orElse(null)); + localPolicies.map(lp -> lp.getLeft().namespaceAntiAffinityGroup).orElse(null), + localPolicies.map(lp -> lp.getLeft().migrated).orElse(false)); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java index f294866095250..18cc449d15dcb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java @@ -102,6 +102,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; +import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.DispatchRate; @@ -2195,4 +2196,30 @@ public void testDispatcherPauseOnAckStatePersistent() throws Exception { admin.namespaces().deleteNamespace(namespace); } + + public void testMigratedInfoIsNotLostDuringOtherLocalPoliciesUpdate() throws Exception { + String namespace = BrokerTestUtil.newUniqueName(this.testTenant + "/namespace"); + admin.namespaces().createNamespace(namespace, Set.of(testLocalCluster)); + + admin.namespaces().updateMigrationState(namespace, true); + assertTrue(admin.namespaces().getPolicies(namespace).migrated); + + String bookieAffinityGroupPrimary = "group1"; + admin.namespaces().setBookieAffinityGroup(namespace, + BookieAffinityGroupData.builder().bookkeeperAffinityGroupPrimary(bookieAffinityGroupPrimary).build()); + assertEquals(admin.namespaces().getBookieAffinityGroup(namespace).getBookkeeperAffinityGroupPrimary(), + bookieAffinityGroupPrimary); + assertTrue(admin.namespaces().getPolicies(namespace).migrated); + + String namespaceAntiAffinityGroup = "group2"; + admin.namespaces().setNamespaceAntiAffinityGroup(namespace, namespaceAntiAffinityGroup); + assertEquals(admin.namespaces().getNamespaceAntiAffinityGroup(namespace), namespaceAntiAffinityGroup); + assertTrue(admin.namespaces().getPolicies(namespace).migrated); + + admin.namespaces().deleteBookieAffinityGroup(namespace); + assertTrue(admin.namespaces().getPolicies(namespace).migrated); + + admin.namespaces().deleteNamespaceAntiAffinityGroup(namespace); + assertTrue(admin.namespaces().getPolicies(namespace).migrated); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/LocalPolicies.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/LocalPolicies.java index 3b17dbe067ebd..43f5130eb9fe8 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/LocalPolicies.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/LocalPolicies.java @@ -34,20 +34,29 @@ public class LocalPolicies { public final BookieAffinityGroupData bookieAffinityGroup; // namespace anti-affinity-group public final String namespaceAntiAffinityGroup; - public boolean migrated; + public final boolean migrated; public LocalPolicies() { bundles = defaultBundle(); bookieAffinityGroup = null; namespaceAntiAffinityGroup = null; + migrated = false; } public LocalPolicies(BundlesData data, BookieAffinityGroupData bookieAffinityGroup, String namespaceAntiAffinityGroup) { + this(data, bookieAffinityGroup, namespaceAntiAffinityGroup, false); + } + + public LocalPolicies(BundlesData data, + BookieAffinityGroupData bookieAffinityGroup, + String namespaceAntiAffinityGroup, + boolean migrated) { bundles = data; this.bookieAffinityGroup = bookieAffinityGroup; this.namespaceAntiAffinityGroup = namespaceAntiAffinityGroup; + this.migrated = migrated; } } \ No newline at end of file From 34c2f30d7838a1d50484985ee8bcfb1d573c50ed Mon Sep 17 00:00:00 2001 From: hrzzzz <64506104+hrzzzz@users.noreply.github.com> Date: Fri, 20 Dec 2024 20:05:09 +0800 Subject: [PATCH 213/327] [fix][client] Fix reader message filtering issue during blue-green cluster switch (#23693) Co-authored-by: ruihongzhou --- .../broker/service/ClusterMigrationTest.java | 111 ++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 7 ++ 2 files changed, 118 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java index e6a7d049366e4..167c154c1fd88 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java @@ -45,6 +45,7 @@ import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl; @@ -911,6 +912,116 @@ public void testNamespaceMigration(SubscriptionType subType, boolean isClusterMi client2.close(); } + public void testMigrationWithReader() throws Exception { + final String topicName = BrokerTestUtil + .newUniqueName("persistent://" + namespace + "/migrationTopic"); + + @Cleanup + PulsarClient client1 = PulsarClient.builder() + .serviceUrl(url1.toString()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + // cluster-1 producer/reader + Producer producer1 = client1.newProducer() + .topic(topicName) + .enableBatching(false) + .producerName("cluster1-1") + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + Reader reader1 =client1.newReader() + .topic(topicName) + .startMessageId(MessageId.earliest) + .subscriptionRolePrefix("s1") + .create(); + + AbstractTopic topic1 = (AbstractTopic) pulsar1.getBrokerService().getTopic(topicName, false).getNow(null).get(); + retryStrategically((test) -> !topic1.getProducers().isEmpty(), 5, 500); + retryStrategically((test) -> !topic1.getSubscriptions().isEmpty(), 5, 500); + assertFalse(topic1.getProducers().isEmpty()); + assertFalse(topic1.getSubscriptions().isEmpty()); + + // build backlog + reader1.close(); + int n = 8; + for (int i = 0; i < n; i++) { + producer1.send("test1".getBytes()); + } + + @Cleanup + PulsarClient client2 = PulsarClient.builder() + .serviceUrl(url2.toString()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + // cluster-2 producer + Producer producer2 = client2.newProducer() + .topic(topicName) + .enableBatching(false) + .producerName("cluster2-1") + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + + AbstractTopic topic2 = (AbstractTopic) pulsar2.getBrokerService().getTopic(topicName, false).getNow(null).get(); + assertFalse(topic2.getProducers().isEmpty()); + assertTrue(topic2.getSubscriptions().isEmpty()); + + // migrate topic to cluster-2 + ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getWebServiceAddress(), pulsar2.getWebServiceAddressTls(), + pulsar2.getBrokerServiceUrl(), null); + admin1.clusters().updateClusterMigration("r1", true, migratedUrl); + assertEquals(admin1.clusters().getClusterMigration("r1").getMigratedClusterUrl(), migratedUrl); + retryStrategically((test) -> { + try { + topic1.checkClusterMigration().get(); + return true; + } catch (Exception e) { + // ok + } + return false; + }, 10, 500); + topic1.checkClusterMigration().get(); + + sleep(1000); + producer1.sendAsync("test1".getBytes()); + + // producer is disconnected from cluster-1 + retryStrategically((test) -> topic1.getProducers().isEmpty(), 10, 500); + assertTrue(topic1.getProducers().isEmpty()); + + // producer is connected with cluster-2 + retryStrategically((test) -> topic2.getProducers().size() == 2, 10, 500); + assertEquals(topic2.getProducers().size(), 2); + + // try to consume backlog messages from cluster-1 + reader1 = client1.newReader() + .topic(topicName) + .startMessageId(MessageId.earliest) + .subscriptionRolePrefix("s1") + .create(); + for (int i = 0; i < n; i++) { + Message msg = reader1.readNext(); + assertEquals(msg.getData(), "test1".getBytes()); + } + + // after consuming all messages, reader should have disconnected from cluster-1 and reconnect with cluster-2 + retryStrategically((test) -> !topic2.getSubscriptions().isEmpty(), 10, 500); + assertFalse(topic2.getSubscriptions().isEmpty()); + assertTrue(topic1.getSubscriptions().isEmpty()); + + n = 4; + // publish messages to cluster-2 and consume them + for (int i = 0; i < n; i++) { + producer1.send("test2".getBytes()); + } + + for (int i = 0; i < n; i++) { + assertEquals(reader1.readNext(2, TimeUnit.SECONDS).getData(), "test2".getBytes()); + } + + client1.close(); + client2.close(); + } + + @Test(dataProvider = "NamespaceMigrationTopicSubscriptionTypes") public void testNamespaceMigrationWithReplicationBacklog(SubscriptionType subType, boolean isClusterMigrate, boolean isNamespaceMigrate) throws Exception { log.info("--- Starting ReplicatorTest::testNamespaceMigrationWithReplicationBacklog ---"); 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 e01c6d4643b08..4d1b51e34db73 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 @@ -37,6 +37,7 @@ import io.opentelemetry.api.common.Attributes; import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -3107,6 +3108,12 @@ boolean isAckReceiptEnabled() { && Commands.peerSupportsAckReceipt(cnx.getRemoteEndpointProtocolVersion()); } + @Override + protected void setRedirectedClusterURI(String serviceUrl, String serviceUrlTls) throws URISyntaxException { + super.setRedirectedClusterURI(serviceUrl, serviceUrlTls); + acknowledgmentsGroupingTracker.flushAndClean(); + } + private static final Logger log = LoggerFactory.getLogger(ConsumerImpl.class); @VisibleForTesting From 217ebfbeaab9a33e648912bfae8ed47e9199d41a Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Fri, 20 Dec 2024 20:19:16 +0800 Subject: [PATCH 214/327] [fix][client] Fix enableRetry for consumers using legacy topic naming where cluster name is included (#23753) --- .../client/impl/TopicsConsumerImplTest.java | 18 ++++++++++++++++++ .../client/impl/ConsumerBuilderImpl.java | 8 ++++---- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java index 1d5ac75962524..3c7cd16f14408 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java @@ -138,6 +138,24 @@ public void testDifferentTopicsNameSubscribe() throws Exception { } } + @Test(timeOut = testTimeout) + public void testRetryClusterTopic() throws Exception { + String key = "testRetryClusterTopic"; + final String topicName = "persistent://prop/use/ns-abc1/topic-1-" + key; + TenantInfoImpl tenantInfo = createDefaultTenantInfo(); + final String namespace = "prop/ns-abc1"; + admin.tenants().createTenant("prop", tenantInfo); + admin.namespaces().createNamespace(namespace, Set.of("test")); + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName("my-sub") + .subscriptionType(SubscriptionType.Shared) + .enableRetry(true) + .ackTimeout(ackTimeOutMillis, TimeUnit.MILLISECONDS) + .subscribe(); + assertTrue(consumer instanceof MultiTopicsConsumerImpl); + } + @Test(timeOut = testTimeout) public void testGetConsumersAndGetTopics() throws Exception { String key = "TopicsConsumerGet"; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java index 351025d426a39..35f772028f17a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java @@ -154,10 +154,10 @@ public CompletableFuture> subscribeAsync() { if (conf.isRetryEnable() && conf.getTopicNames().size() > 0) { TopicName topicFirst = TopicName.get(conf.getTopicNames().iterator().next()); //Issue 9327: do compatibility check in case of the default retry and dead letter topic name changed - String oldRetryLetterTopic = topicFirst.getNamespace() + "/" + conf.getSubscriptionName() - + RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX; - String oldDeadLetterTopic = topicFirst.getNamespace() + "/" + conf.getSubscriptionName() - + RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; + String oldRetryLetterTopic = TopicName.get(topicFirst.getDomain().value(), topicFirst.getNamespaceObject(), + conf.getSubscriptionName() + RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX).toString(); + String oldDeadLetterTopic = TopicName.get(topicFirst.getDomain().value(), topicFirst.getNamespaceObject(), + conf.getSubscriptionName() + RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX).toString(); DeadLetterPolicy deadLetterPolicy = conf.getDeadLetterPolicy(); if (deadLetterPolicy == null || StringUtils.isBlank(deadLetterPolicy.getRetryLetterTopic()) || StringUtils.isBlank(deadLetterPolicy.getDeadLetterTopic())) { From 3fce3097c76a9c8cb64cf3d8d87f6e050e6cb3a5 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 20 Dec 2024 20:58:03 +0800 Subject: [PATCH 215/327] [improve][client] Make replicateSubscriptionState nullable (#23757) Signed-off-by: Zixuan Liu --- .../pulsar/broker/service/ServerCnx.java | 4 +- .../broker/service/SubscriptionOption.java | 2 +- .../nonpersistent/NonPersistentTopic.java | 4 +- .../persistent/PersistentSubscription.java | 19 +++- .../service/persistent/PersistentTopic.java | 19 ++-- .../client/api/ReplicateSubscriptionTest.java | 96 +++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 2 +- .../impl/conf/ConsumerConfigurationData.java | 14 ++- .../client/impl/ConsumerBuilderImplTest.java | 36 ++++++- .../pulsar/common/protocol/Commands.java | 8 +- 10 files changed, 176 insertions(+), 28 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/ReplicateSubscriptionTest.java 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 f9e593345d85f..2415930a99a6c 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 @@ -1241,8 +1241,8 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { ? subscribe.getStartMessageRollbackDurationSec() : -1; final SchemaData schema = subscribe.hasSchema() ? getSchema(subscribe.getSchema()) : null; - final boolean isReplicated = subscribe.hasReplicateSubscriptionState() - && subscribe.isReplicateSubscriptionState(); + final Boolean isReplicated = + subscribe.hasReplicateSubscriptionState() ? subscribe.isReplicateSubscriptionState() : null; final boolean forceTopicCreation = subscribe.isForceTopicCreation(); final KeySharedMeta keySharedMeta = subscribe.hasKeySharedMeta() ? new KeySharedMeta().copyFrom(subscribe.getKeySharedMeta()) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SubscriptionOption.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SubscriptionOption.java index af56d023616b4..328e7618f8cd8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SubscriptionOption.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SubscriptionOption.java @@ -46,7 +46,7 @@ public class SubscriptionOption { private boolean readCompacted; private CommandSubscribe.InitialPosition initialPosition; private long startMessageRollbackDurationSec; - private boolean replicatedSubscriptionStateArg; + private Boolean replicatedSubscriptionStateArg; private KeySharedMeta keySharedMeta; private Optional> subscriptionProperties; private long consumerEpoch; 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 34c2678f847a5..7cdc8cc11a482 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 @@ -256,7 +256,7 @@ public CompletableFuture subscribe(SubscriptionOption option) { return internalSubscribe(option.getCnx(), option.getSubscriptionName(), option.getConsumerId(), option.getSubType(), option.getPriorityLevel(), option.getConsumerName(), option.getStartMessageId(), option.getMetadata(), option.isReadCompacted(), - option.getStartMessageRollbackDurationSec(), option.isReplicatedSubscriptionStateArg(), + option.getStartMessageRollbackDurationSec(), option.getReplicatedSubscriptionStateArg(), option.getKeySharedMeta(), option.getSubscriptionProperties().orElse(null), option.getSchemaType()); } @@ -279,7 +279,7 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St String consumerName, MessageId startMessageId, Map metadata, boolean readCompacted, long resetStartMessageBackInSec, - boolean replicateSubscriptionState, + Boolean replicateSubscriptionState, KeySharedMeta keySharedMeta, Map subscriptionProperties, SchemaType schemaType) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 0096f398ada91..8cebbd52695a8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -133,9 +133,11 @@ public class PersistentSubscription extends AbstractSubscription { private volatile Map subscriptionProperties; private volatile CompletableFuture fenceFuture; private volatile CompletableFuture inProgressResetCursorFuture; + private volatile Boolean replicatedControlled; - static Map getBaseCursorProperties(boolean isReplicated) { - return isReplicated ? REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES : NON_REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES; + static Map getBaseCursorProperties(Boolean isReplicated) { + return isReplicated != null && isReplicated ? REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES : + NON_REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES; } static boolean isCursorFromReplicatedSubscription(ManagedCursor cursor) { @@ -143,19 +145,21 @@ static boolean isCursorFromReplicatedSubscription(ManagedCursor cursor) { } public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, - boolean replicated) { + Boolean replicated) { this(topic, subscriptionName, cursor, replicated, Collections.emptyMap()); } public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, - boolean replicated, Map subscriptionProperties) { + Boolean replicated, Map subscriptionProperties) { this.topic = topic; this.cursor = cursor; this.topicName = topic.getName(); this.subName = subscriptionName; this.fullName = MoreObjects.toStringHelper(this).add("topic", topicName).add("name", subName).toString(); this.expiryMonitor = new PersistentMessageExpiryMonitor(topic, subscriptionName, cursor, this); - this.setReplicated(replicated); + if (replicated != null) { + this.setReplicated(replicated); + } this.subscriptionProperties = MapUtils.isEmpty(subscriptionProperties) ? Collections.emptyMap() : Collections.unmodifiableMap(subscriptionProperties); if (topic.getBrokerService().getPulsar().getConfig().isTransactionCoordinatorEnabled() @@ -194,6 +198,7 @@ public boolean isReplicated() { } public boolean setReplicated(boolean replicated) { + replicatedControlled = replicated; ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); if (!replicated || !config.isEnableReplicatedSubscriptions()) { @@ -1557,4 +1562,8 @@ public PositionInPendingAckStats checkPositionInPendingAckState(Position positio private static final Logger log = LoggerFactory.getLogger(PersistentSubscription.class); + @VisibleForTesting + public Boolean getReplicatedControlled() { + return replicatedControlled; + } } 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 056fad2a005b4..11220d1c955e3 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 @@ -513,7 +513,7 @@ private void createPersistentSubscriptions() { } else { final String subscriptionName = Codec.decode(cursor.getName()); subscriptions.put(subscriptionName, createPersistentSubscription(subscriptionName, cursor, - PersistentSubscription.isCursorFromReplicatedSubscription(cursor), + PersistentSubscription.isCursorFromReplicatedSubscription(cursor) ? true : null, cursor.getCursorProperties())); // subscription-cursor gets activated by default: deactivate as there is no active subscription // right now @@ -584,7 +584,7 @@ public CompletableFuture unloadSubscription(@Nonnull String subName) { } private PersistentSubscription createPersistentSubscription(String subscriptionName, ManagedCursor cursor, - boolean replicated, Map subscriptionProperties) { + Boolean replicated, Map subscriptionProperties) { requireNonNull(topicCompactionService); if (isCompactionSubscription(subscriptionName) && topicCompactionService instanceof PulsarTopicCompactionService pulsarTopicCompactionService) { @@ -888,7 +888,7 @@ public CompletableFuture subscribe(SubscriptionOption option) { option.getSubType(), option.getPriorityLevel(), option.getConsumerName(), option.isDurable(), option.getStartMessageId(), option.getMetadata(), option.isReadCompacted(), option.getInitialPosition(), option.getStartMessageRollbackDurationSec(), - option.isReplicatedSubscriptionStateArg(), option.getKeySharedMeta(), + option.getReplicatedSubscriptionStateArg(), option.getKeySharedMeta(), option.getSubscriptionProperties().orElse(Collections.emptyMap()), option.getConsumerEpoch(), option.getSchemaType()); } @@ -900,7 +900,7 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St Map metadata, boolean readCompacted, InitialPosition initialPosition, long startMessageRollbackDurationSec, - boolean replicatedSubscriptionStateArg, + Boolean replicatedSubscriptionStateArg, KeySharedMeta keySharedMeta, Map subscriptionProperties, long consumerEpoch, @@ -911,12 +911,9 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St } return brokerService.checkTopicNsOwnership(getName()).thenCompose(__ -> { - boolean replicatedSubscriptionState = replicatedSubscriptionStateArg; - - if (replicatedSubscriptionState + if (replicatedSubscriptionStateArg != null && replicatedSubscriptionStateArg && !brokerService.pulsar().getConfiguration().isEnableReplicatedSubscriptions()) { log.warn("[{}] Replicated Subscription is disabled by broker.", getName()); - replicatedSubscriptionState = false; } if (subType == SubType.Key_Shared @@ -985,7 +982,7 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St CompletableFuture subscriptionFuture = isDurable ? getDurableSubscription(subscriptionName, initialPosition, startMessageRollbackDurationSec, - replicatedSubscriptionState, subscriptionProperties) + replicatedSubscriptionStateArg, subscriptionProperties) : getNonDurableSubscription(subscriptionName, startMessageId, initialPosition, startMessageRollbackDurationSec, readCompacted, subscriptionProperties); @@ -1082,7 +1079,7 @@ public CompletableFuture subscribe(final TransportCnx cnx, String subs private CompletableFuture getDurableSubscription(String subscriptionName, InitialPosition initialPosition, long startMessageRollbackDurationSec, - boolean replicated, + Boolean replicated, Map subscriptionProperties) { CompletableFuture subscriptionFuture = new CompletableFuture<>(); if (checkMaxSubscriptionsPerTopicExceed(subscriptionName)) { @@ -1113,7 +1110,7 @@ public void openCursorComplete(ManagedCursor cursor, Object ctx) { return; } } - if (replicated && !subscription.isReplicated()) { + if (replicated != null && replicated && !subscription.isReplicated()) { // Flip the subscription state subscription.setReplicated(replicated); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ReplicateSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ReplicateSubscriptionTest.java new file mode 100644 index 0000000000000..327081bf1b9c8 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ReplicateSubscriptionTest.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.client.impl.ConsumerBuilderImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +public class ReplicateSubscriptionTest extends ProducerConsumerBase { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + } + + @DataProvider + public Object[] replicateSubscriptionState() { + return new Object[]{ + Boolean.TRUE, + Boolean.FALSE, + null + }; + } + + @Test(dataProvider = "replicateSubscriptionState") + public void testReplicateSubscriptionState(Boolean replicateSubscriptionState) + throws Exception { + String topic = "persistent://my-property/my-ns/" + System.nanoTime(); + String subName = "sub-" + System.nanoTime(); + ConsumerBuilder consumerBuilder = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName(subName); + if (replicateSubscriptionState != null) { + consumerBuilder.replicateSubscriptionState(replicateSubscriptionState); + } + ConsumerBuilderImpl consumerBuilderImpl = (ConsumerBuilderImpl) consumerBuilder; + assertEquals(consumerBuilderImpl.getConf().getReplicateSubscriptionState(), replicateSubscriptionState); + @Cleanup + Consumer ignored = consumerBuilder.subscribe(); + CompletableFuture> topicIfExists = pulsar.getBrokerService().getTopicIfExists(topic); + assertThat(topicIfExists) + .succeedsWithin(3, TimeUnit.SECONDS) + .matches(optionalTopic -> { + assertTrue(optionalTopic.isPresent()); + Topic topicRef = optionalTopic.get(); + Subscription subscription = topicRef.getSubscription(subName); + assertNotNull(subscription); + assertTrue(subscription instanceof PersistentSubscription); + PersistentSubscription persistentSubscription = (PersistentSubscription) subscription; + assertEquals(persistentSubscription.getReplicatedControlled(), replicateSubscriptionState); + return true; + }); + } +} 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 4d1b51e34db73..16dc70f736e7d 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 @@ -902,7 +902,7 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { synchronized (this) { ByteBuf request = Commands.newSubscribe(topic, subscription, consumerId, requestId, getSubType(), priorityLevel, consumerName, isDurable, startMessageIdData, metadata, readCompacted, - conf.isReplicateSubscriptionState(), + conf.getReplicateSubscriptionState(), InitialPosition.valueOf(subscriptionInitialPosition.getValue()), startMessageRollbackDuration, si, createTopicIfDoesNotExist, conf.getKeySharedPolicy(), // Use the current epoch to subscribe. diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index f9ff5913f62da..6e884ba279116 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; import io.swagger.annotations.ApiModelProperty; import java.io.Serializable; @@ -381,7 +382,8 @@ public int getMaxPendingChuckedMessage() { value = "If `replicateSubscriptionState` is enabled, a subscription state is replicated to geo-replicated" + " clusters." ) - private boolean replicateSubscriptionState = false; + @JsonProperty(access = JsonProperty.Access.READ_WRITE) + private Boolean replicateSubscriptionState; private boolean resetIncludeHead = false; @@ -437,4 +439,14 @@ public ConsumerConfigurationData clone() { throw new RuntimeException("Failed to clone ConsumerConfigurationData"); } } + + /** + * Backward compatibility with the old `replicateSubscriptionState` field. + * @deprecated Using {@link #getReplicateSubscriptionState()} instead. + */ + @JsonIgnore + @Deprecated + public boolean isReplicateSubscriptionState() { + return replicateSubscriptionState != null && replicateSubscriptionState; + } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerBuilderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerBuilderImplTest.java index e4b7b4d1ec85e..c103712d40055 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerBuilderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerBuilderImplTest.java @@ -504,7 +504,7 @@ public void testLoadConf() throws Exception { assertTrue(configurationData.isRetryEnable()); assertFalse(configurationData.isAutoUpdatePartitions()); assertEquals(configurationData.getAutoUpdatePartitionsIntervalSeconds(), 2); - assertTrue(configurationData.isReplicateSubscriptionState()); + assertEquals(configurationData.getReplicateSubscriptionState(), Boolean.TRUE); assertTrue(configurationData.isResetIncludeHead()); assertTrue(configurationData.isBatchIndexAckEnabled()); assertTrue(configurationData.isAckReceiptEnabled()); @@ -564,7 +564,7 @@ public void testLoadConfNotModified() { assertFalse(configurationData.isRetryEnable()); assertTrue(configurationData.isAutoUpdatePartitions()); assertEquals(configurationData.getAutoUpdatePartitionsIntervalSeconds(), 60); - assertFalse(configurationData.isReplicateSubscriptionState()); + assertNull(configurationData.getReplicateSubscriptionState()); assertFalse(configurationData.isResetIncludeHead()); assertFalse(configurationData.isBatchIndexAckEnabled()); assertFalse(configurationData.isAckReceiptEnabled()); @@ -584,6 +584,38 @@ public void testLoadConfNotModified() { assertNull(configurationData.getPayloadProcessor()); } + @Test + public void testReplicateSubscriptionState() { + ConsumerBuilderImpl consumerBuilder = createConsumerBuilder(); + assertNull(consumerBuilder.getConf().getReplicateSubscriptionState()); + + consumerBuilder.replicateSubscriptionState(true); + assertEquals(consumerBuilder.getConf().getReplicateSubscriptionState(), Boolean.TRUE); + + consumerBuilder.replicateSubscriptionState(false); + assertEquals(consumerBuilder.getConf().getReplicateSubscriptionState(), Boolean.FALSE); + + Map conf = new HashMap<>(); + consumerBuilder = createConsumerBuilder(); + consumerBuilder.loadConf(conf); + assertNull(consumerBuilder.getConf().getReplicateSubscriptionState()); + + conf.put("replicateSubscriptionState", true); + consumerBuilder = createConsumerBuilder(); + consumerBuilder.loadConf(conf); + assertEquals(consumerBuilder.getConf().getReplicateSubscriptionState(), Boolean.TRUE); + + conf.put("replicateSubscriptionState", false); + consumerBuilder = createConsumerBuilder(); + consumerBuilder.loadConf(conf); + assertEquals(consumerBuilder.getConf().getReplicateSubscriptionState(), Boolean.FALSE); + + conf.put("replicateSubscriptionState", null); + consumerBuilder = createConsumerBuilder(); + consumerBuilder.loadConf(conf); + assertNull(consumerBuilder.getConf().getReplicateSubscriptionState()); + } + private ConsumerBuilderImpl createConsumerBuilder() { ConsumerBuilderImpl consumerBuilder = new ConsumerBuilderImpl<>(null, Schema.BYTES); Map properties = new HashMap<>(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 19aa9907549d9..4f390cc99e610 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -583,7 +583,7 @@ public static ByteBuf newSubscribe(String topic, String subscription, long consu public static ByteBuf newSubscribe(String topic, String subscription, long consumerId, long requestId, SubType subType, int priorityLevel, String consumerName, boolean isDurable, MessageIdData startMessageId, - Map metadata, boolean readCompacted, boolean isReplicated, + Map metadata, boolean readCompacted, Boolean isReplicated, InitialPosition subscriptionInitialPosition, long startMessageRollbackDurationInSec, SchemaInfo schemaInfo, boolean createTopicIfDoesNotExist) { return newSubscribe(topic, subscription, consumerId, requestId, subType, priorityLevel, consumerName, @@ -594,7 +594,7 @@ public static ByteBuf newSubscribe(String topic, String subscription, long consu public static ByteBuf newSubscribe(String topic, String subscription, long consumerId, long requestId, SubType subType, int priorityLevel, String consumerName, boolean isDurable, MessageIdData startMessageId, - Map metadata, boolean readCompacted, boolean isReplicated, + Map metadata, boolean readCompacted, Boolean isReplicated, InitialPosition subscriptionInitialPosition, long startMessageRollbackDurationInSec, SchemaInfo schemaInfo, boolean createTopicIfDoesNotExist, KeySharedPolicy keySharedPolicy, Map subscriptionProperties, long consumerEpoch) { @@ -610,9 +610,11 @@ public static ByteBuf newSubscribe(String topic, String subscription, long consu .setDurable(isDurable) .setReadCompacted(readCompacted) .setInitialPosition(subscriptionInitialPosition) - .setReplicateSubscriptionState(isReplicated) .setForceTopicCreation(createTopicIfDoesNotExist) .setConsumerEpoch(consumerEpoch); + if (isReplicated != null) { + subscribe.setReplicateSubscriptionState(isReplicated); + } if (subscriptionProperties != null && !subscriptionProperties.isEmpty()) { List keyValues = new ArrayList<>(); From 906d10e547abf8097dfd16e5564df3573246f6fc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 21 Dec 2024 00:41:37 -0800 Subject: [PATCH 216/327] [fix] Fix issues with Pulsar Alpine docker image stability: remove glibc-compat (#23762) --- docker/glibc-package/Dockerfile | 80 ---------- docker/glibc-package/scripts/APKBUILD | 53 ------- .../glibc-package/scripts/glibc-bin.trigger | 21 --- docker/glibc-package/scripts/ld.so.conf | 23 --- docker/kinesis-producer-alpine/Dockerfile | 90 ++++++++++++ .../README.md | 14 +- .../kinesis-producer-alpine/build-alpine.sh | 139 ++++++++++++++++++ .../kinesis_producer_alpine.patch | 127 ++++++++++++++++ docker/pulsar-all/Dockerfile | 38 ++++- docker/pulsar-all/pom.xml | 5 + docker/pulsar/Dockerfile | 11 +- pom.xml | 3 +- pulsar-io/kinesis/pom.xml | 2 +- .../apache/pulsar/io/kinesis/KinesisSink.java | 8 +- .../pulsar/io/kinesis/KinesisSinkConfig.java | 23 +++ .../pulsar/io/kinesis/KinesisSinkTest.java | 37 ++--- .../latest-version-image/Dockerfile | 24 +-- .../latest-version-image/pom.xml | 1 - .../io/sinks/KinesisSinkTester.java | 13 +- 19 files changed, 469 insertions(+), 243 deletions(-) delete mode 100644 docker/glibc-package/Dockerfile delete mode 100644 docker/glibc-package/scripts/APKBUILD delete mode 100755 docker/glibc-package/scripts/glibc-bin.trigger delete mode 100644 docker/glibc-package/scripts/ld.so.conf create mode 100644 docker/kinesis-producer-alpine/Dockerfile rename docker/{glibc-package => kinesis-producer-alpine}/README.md (54%) create mode 100644 docker/kinesis-producer-alpine/build-alpine.sh create mode 100644 docker/kinesis-producer-alpine/kinesis_producer_alpine.patch diff --git a/docker/glibc-package/Dockerfile b/docker/glibc-package/Dockerfile deleted file mode 100644 index 016e5c622365f..0000000000000 --- a/docker/glibc-package/Dockerfile +++ /dev/null @@ -1,80 +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. -# - - -ARG GLIBC_VERSION=2.38 -ARG ALPINE_VERSION=3.20 - -FROM ubuntu:22.04 as build -ARG GLIBC_VERSION - -RUN apt-get -q update \ - && apt-get -qy install \ - bison \ - build-essential \ - gawk \ - gettext \ - openssl \ - python3 \ - texinfo \ - wget - -# Build GLibc -RUN wget -qO- https://ftpmirror.gnu.org/libc/glibc-${GLIBC_VERSION}.tar.gz | tar zxf - -RUN mkdir /glibc-build -WORKDIR /glibc-build -RUN /glibc-${GLIBC_VERSION}/configure \ - --prefix=/usr/glibc-compat \ - --libdir=/usr/glibc-compat/lib \ - --libexecdir=/usr/glibc-compat/lib \ - --enable-multi-arch \ - --enable-stack-protector=strong -RUN make -j$(nproc) -RUN make install -RUN tar --dereference --hard-dereference -zcf /glibc-bin.tar.gz /usr/glibc-compat - - -################################################ -## Build the APK package -FROM alpine:$ALPINE_VERSION as apk -ARG GLIBC_VERSION - -RUN apk add abuild sudo build-base - -RUN mkdir /build -WORKDIR build - -COPY --from=build /glibc-bin.tar.gz /build - -COPY ./scripts /build - -RUN echo "pkgver=\"${GLIBC_VERSION}\"" >> /build/APKBUILD -RUN echo "sha512sums=\"$(sha512sum glibc-bin.tar.gz ld.so.conf)\"" >> /build/APKBUILD - -RUN abuild-keygen -a -i -n -RUN abuild -F -c -r - -################################################ -## Last stage - Only leaves the packages -FROM busybox -ARG GLIBC_VERSION - -RUN mkdir -p /root/packages -COPY --from=apk /root/packages/*/glibc-${GLIBC_VERSION}-r0.apk /root/packages -COPY --from=apk /root/packages/*/glibc-bin-${GLIBC_VERSION}-r0.apk /root/packages diff --git a/docker/glibc-package/scripts/APKBUILD b/docker/glibc-package/scripts/APKBUILD deleted file mode 100644 index 0545508f0a7d4..0000000000000 --- a/docker/glibc-package/scripts/APKBUILD +++ /dev/null @@ -1,53 +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. -# - -pkgname="glibc" -pkgrel="0" -pkgdesc="GNU C Library compatibility layer" -arch="all" -url="https:/pulsar.apache.org" -license="LGPL" -options="!check" -source="glibc-bin.tar.gz -ld.so.conf" -subpackages="${pkgname}-bin ${pkgname}-dev" -triggers="glibc-bin.trigger=/lib:/usr/lib:/usr/glibc-compat/lib" -depends="libuuid libgcc" - -package() { - mkdir -p $pkgdir/lib $pkgdir/usr/glibc-compat/lib/locale $pkgdir/usr/glibc-compat/lib64 $pkgdir/etc $pkgdir/usr/glibc-compat/etc/ - cp -a $srcdir/usr $pkgdir - cp $srcdir/ld.so.conf $pkgdir/usr/glibc-compat/etc/ld.so.conf - cd $pkgdir/usr/glibc-compat - rm -rf etc/rpc bin sbin lib/gconv lib/getconf lib/audit share var include - - FILENAME=$(ls $pkgdir/usr/glibc-compat/lib/ld-linux-*.so.*) - LIBNAME=$(basename $FILENAME) - ln -s /usr/glibc-compat/lib/$LIBNAME $pkgdir/lib/$LIBNAME - ln -s /usr/glibc-compat/lib/$LIBNAME $pkgdir/usr/glibc-compat/lib64/$LIBNAME - ln -s /usr/glibc-compat/etc/ld.so.cache $pkgdir/etc/ld.so.cache -} - -bin() { - depends="$pkgname libc6-compat" - mkdir -p $subpkgdir/usr/glibc-compat - cp -a $srcdir/usr/glibc-compat/bin $subpkgdir/usr/glibc-compat - cp -a $srcdir/usr/glibc-compat/sbin $subpkgdir/usr/glibc-compat -} - diff --git a/docker/glibc-package/scripts/glibc-bin.trigger b/docker/glibc-package/scripts/glibc-bin.trigger deleted file mode 100755 index 5bae5d7ca2bda..0000000000000 --- a/docker/glibc-package/scripts/glibc-bin.trigger +++ /dev/null @@ -1,21 +0,0 @@ -#!/bin/sh -# -# 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. -# - -/usr/glibc-compat/sbin/ldconfig \ No newline at end of file diff --git a/docker/glibc-package/scripts/ld.so.conf b/docker/glibc-package/scripts/ld.so.conf deleted file mode 100644 index 6548b9300bb9c..0000000000000 --- a/docker/glibc-package/scripts/ld.so.conf +++ /dev/null @@ -1,23 +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. -# - -/usr/local/lib -/usr/glibc-compat/lib -/usr/lib -/lib diff --git a/docker/kinesis-producer-alpine/Dockerfile b/docker/kinesis-producer-alpine/Dockerfile new file mode 100644 index 0000000000000..ffdf44f55d083 --- /dev/null +++ b/docker/kinesis-producer-alpine/Dockerfile @@ -0,0 +1,90 @@ +# +# 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. +# + +ARG ALPINE_VERSION=3.20 + +# Builds an Alpine image with kinesis_producer compiled for Alpine Linux / musl + +# Build stage +FROM alpine:$ALPINE_VERSION AS kinesis-producer-build +ENV KINESIS_PRODUCER_LIB_VERSION=0.15.12 + +# Install build dependencies +RUN apk update && apk add --no-cache \ + git \ + binutils \ + coreutils \ + alpine-sdk \ + util-linux \ + cmake \ + autoconf \ + automake \ + libtool \ + curl \ + bash \ + tar \ + libuuid \ + linux-headers \ + zlib \ + zlib-dev \ + perl \ + wget \ + boost-dev \ + openssl-dev \ + curl-dev \ + build-base \ + util-linux-dev \ + g++ \ + make \ + upx + +ENV LANG=C.UTF-8 + +RUN mkdir /build +COPY kinesis_producer_alpine.patch /build/ + +# Clone KPL and copy build script +RUN cd /build && \ + git clone --depth 1 --single-branch --branch v${KINESIS_PRODUCER_LIB_VERSION} https://github.com/awslabs/amazon-kinesis-producer && \ + cd amazon-kinesis-producer && \ + git apply ../kinesis_producer_alpine.patch + +# Copy and execute build script +COPY build-alpine.sh /build/ +RUN chmod +x /build/build-alpine.sh +RUN /build/build-alpine.sh + +# Final stage +FROM alpine:$ALPINE_VERSION +COPY --from=kinesis-producer-build /opt/amazon-kinesis-producer /opt/amazon-kinesis-producer +RUN apk update && apk add --no-cache \ + brotli-libs \ + c-ares \ + libcrypto3 \ + libcurl \ + libgcc \ + libidn2 \ + libpsl \ + libssl3 \ + libunistring \ + nghttp2-libs \ + zlib \ + zstd-libs \ + libuuid +WORKDIR /opt/amazon-kinesis-producer/bin diff --git a/docker/glibc-package/README.md b/docker/kinesis-producer-alpine/README.md similarity index 54% rename from docker/glibc-package/README.md rename to docker/kinesis-producer-alpine/README.md index ee1f643705ad2..4526f08c65ec1 100644 --- a/docker/glibc-package/README.md +++ b/docker/kinesis-producer-alpine/README.md @@ -19,21 +19,19 @@ --> -# GLibc compatibility package +# Alpine image with kinesis_producer compiled for Alpine Linux / musl -This directory includes the Docker scripts to build an image with GLibc compiled for Alpine Linux. +This directory includes the Docker scripts to build an image with `kinesis_producer` for Alpine Linux. +`kinesis_producer` is a native executable that is required by [Amazon Kinesis Producer library (KPL)](https://github.com/awslabs/amazon-kinesis-producer) which is used by the Pulsar IO Kinesis Sink connector. The default `kinesis_producer` binary is compiled for glibc, and it does not work on Alpine Linux which uses musl. -This is used to ensure plugins that are going to be used in the Pulsar image and that are depeding on GLibc, will -still be working correctly in the Alpine Image. (eg: Netty Tc-Native and Kinesis Producer Library). - -This image only needs to be re-created when we want to upgrade to a newer version of GLibc. +This image only needs to be re-created when we want to upgrade to a newer version of `kinesis_producer`. # Steps 1. Change the version in the Dockerfile for this directory. 2. Rebuild the image and push it to Docker Hub: ``` -docker buildx build --platform=linux/amd64,linux/arm64 -t apachepulsar/glibc-base:2.38 . --push +docker buildx build --platform=linux/amd64,linux/arm64 -t apachepulsar/pulsar-io-kinesis-sink-kinesis_producer:0.15.12 . --push ``` -The image tag is then used in `docker/pulsar/Dockerfile`. +The image tag is then used in `docker/pulsar-all/Dockerfile`. The `kinesis_producer` binary is copied from the image to the `pulsar-all` image that is used by Pulsar Functions to run the Pulsar IO Kinesis Sink connector. The environment variable `PULSAR_IO_KINESIS_KPL_PATH` is set to `/opt/amazon-kinesis-producer/bin/kinesis_producer` and this is how the Kinesis Sink connector knows where to find the `kinesis_producer` binary. \ No newline at end of file diff --git a/docker/kinesis-producer-alpine/build-alpine.sh b/docker/kinesis-producer-alpine/build-alpine.sh new file mode 100644 index 0000000000000..23718450bbc83 --- /dev/null +++ b/docker/kinesis-producer-alpine/build-alpine.sh @@ -0,0 +1,139 @@ +#!/bin/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 +set -x + +INSTALL_DIR=/build/third_party +AWS_SDK_CPP_VERSION="1.11.420" +PROTOBUF_VERSION="3.11.4" +BOOST_VERSION="1.76.0" +BOOST_VERSION_UNDERSCORED="${BOOST_VERSION//\./_}" + +# Create install directory +mkdir -p $INSTALL_DIR + +# Setup environment variables +export CC="gcc" +export CXX="g++" +export CXXFLAGS="-I$INSTALL_DIR/include -O3 -Wno-implicit-fallthrough -Wno-int-in-bool-context" +export LDFLAGS="-L$INSTALL_DIR/lib" +export LD_LIBRARY_PATH="$INSTALL_DIR/lib:$LD_LIBRARY_PATH" + +cd $INSTALL_DIR + +# Build protobuf +if [ ! -d "protobuf-${PROTOBUF_VERSION}" ]; then + curl -LO https://github.com/protocolbuffers/protobuf/releases/download/v${PROTOBUF_VERSION}/protobuf-all-${PROTOBUF_VERSION}.tar.gz + tar xf protobuf-all-${PROTOBUF_VERSION}.tar.gz + rm protobuf-all-${PROTOBUF_VERSION}.tar.gz + + cd protobuf-${PROTOBUF_VERSION} + ./configure --prefix=${INSTALL_DIR} \ + --disable-shared \ + CFLAGS="-fPIC" \ + CXXFLAGS="-fPIC ${CXXFLAGS}" \ + --with-pic + make -j4 + make install + cd .. +fi + +# Build Boost +if [ ! -d "boost_${BOOST_VERSION_UNDERSCORED}" ]; then + curl -LO https://boostorg.jfrog.io/artifactory/main/release/${BOOST_VERSION}/source/boost_${BOOST_VERSION_UNDERSCORED}.tar.gz + tar xf boost_${BOOST_VERSION_UNDERSCORED}.tar.gz + rm boost_${BOOST_VERSION_UNDERSCORED}.tar.gz + + cd boost_${BOOST_VERSION_UNDERSCORED} + + BOOST_LIBS="regex,thread,log,system,random,filesystem,chrono,atomic,date_time,program_options,test" + + ./bootstrap.sh --with-libraries=$BOOST_LIBS --with-toolset=gcc + + ./b2 \ + -j4 \ + variant=release \ + link=static \ + threading=multi \ + runtime-link=static \ + --prefix=${INSTALL_DIR} \ + cxxflags="-fPIC ${CXXFLAGS}" \ + install + + cd .. +fi + +# Download and build AWS SDK +if [ ! -d "aws-sdk-cpp" ]; then + git clone --depth 1 --branch ${AWS_SDK_CPP_VERSION} https://github.com/awslabs/aws-sdk-cpp.git aws-sdk-cpp + pushd aws-sdk-cpp + git config submodule.fetchJobs 8 + git submodule update --init --depth 1 --recursive + popd + + rm -rf aws-sdk-cpp-build + mkdir aws-sdk-cpp-build + cd aws-sdk-cpp-build + + cmake \ + -DBUILD_ONLY="kinesis;monitoring;sts" \ + -DCMAKE_BUILD_TYPE=RelWithDebInfo \ + -DSTATIC_LINKING=1 \ + -DCMAKE_PREFIX_PATH="$INSTALL_DIR" \ + -DCMAKE_C_COMPILER="$CC" \ + -DCMAKE_CXX_COMPILER="$CXX" \ + -DCMAKE_CXX_FLAGS="$CXXFLAGS" \ + -DCMAKE_INSTALL_PREFIX="$INSTALL_DIR" \ + -DCMAKE_FIND_FRAMEWORK=LAST \ + -DENABLE_TESTING="OFF" \ + ../aws-sdk-cpp + make -j4 + make install + cd .. +fi + +# Build the native kinesis producer +cd /build/amazon-kinesis-producer +ln -fs ../third_party +cmake -DCMAKE_PREFIX_PATH="$INSTALL_DIR" -DCMAKE_BUILD_TYPE=RelWithDebInfo . +make -j4 + +FINAL_DIR=/opt/amazon-kinesis-producer +# copy the binary +mkdir -p $FINAL_DIR/bin +cp kinesis_producer $FINAL_DIR/bin/kinesis_producer.original + +# capture version information +git describe --long --tags > $FINAL_DIR/bin/.version +git rev-parse HEAD > $FINAL_DIR/bin/.revision +uname -a > $FINAL_DIR/bin/.system_info +cat /etc/os-release > $FINAL_DIR/bin/.os_info +date > $FINAL_DIR/bin/.build_time + +# copy tests +mkdir -p $FINAL_DIR/tests +cp tests $FINAL_DIR/tests/ +cp test_driver $FINAL_DIR/tests/ + +# Strip and compress the binary +cd $FINAL_DIR/bin +strip -o kinesis_producer.stripped kinesis_producer.original +upx --best -o kinesis_producer kinesis_producer.stripped \ No newline at end of file diff --git a/docker/kinesis-producer-alpine/kinesis_producer_alpine.patch b/docker/kinesis-producer-alpine/kinesis_producer_alpine.patch new file mode 100644 index 0000000000000..d1ddd6a85501c --- /dev/null +++ b/docker/kinesis-producer-alpine/kinesis_producer_alpine.patch @@ -0,0 +1,127 @@ +From 96ba2eb7145363586529e6c770dcc0920bf04ac2 Mon Sep 17 00:00:00 2001 +From: Lari Hotari +Date: Wed, 18 Dec 2024 17:16:02 +0200 +Subject: [PATCH] Adapt build for Alpine, fix issue with NULL_BACKTRACE support + +- also use dynamic linking to some libraries (zlib, openssl, libz, libcurl, libcrypto) + to reduce binary size +--- + CMakeLists.txt | 20 +++++++++++--------- + aws/utils/backtrace/bsd_backtrace.cc | 2 +- + aws/utils/backtrace/gcc_backtrace.cc | 3 +-- + aws/utils/backtrace/null_backtrace.cc | 5 ++--- + aws/utils/signal_handler.cc | 1 - + 5 files changed, 15 insertions(+), 16 deletions(-) + +diff --git a/CMakeLists.txt b/CMakeLists.txt +index 2dd7084..2ba47e6 100644 +--- a/CMakeLists.txt ++++ b/CMakeLists.txt +@@ -213,22 +213,24 @@ set(STATIC_LIBS + boost_chrono) + + find_package(Threads) +-find_package(ZLIB) ++find_package(ZLIB REQUIRED) + find_package(AWSSDK REQUIRED COMPONENTS kinesis monitoring sts) ++find_package(OpenSSL REQUIRED) ++find_package(CURL REQUIRED) + +-add_library(LibCrypto STATIC IMPORTED) +-set_property(TARGET LibCrypto PROPERTY IMPORTED_LOCATION ${THIRD_PARTY_LIB_DIR}/libcrypto.a) ++add_library(LibCrypto SHARED IMPORTED) ++set_property(TARGET LibCrypto PROPERTY IMPORTED_LOCATION ${OPENSSL_CRYPTO_LIBRARY}) + set_property(TARGET LibCrypto PROPERTY IMPORTED_LINK_INTERFACE_LIBRARIES ${LIBDL_LIBRARIES}) + +-add_library(LibSsl STATIC IMPORTED) +-set_property(TARGET LibSsl PROPERTY IMPORTED_LOCATION ${THIRD_PARTY_LIB_DIR}/libssl.a) ++add_library(LibSsl SHARED IMPORTED) ++set_property(TARGET LibSsl PROPERTY IMPORTED_LOCATION ${OPENSSL_SSL_LIBRARY}) + set_property(TARGET LibSsl PROPERTY IMPORTED_LINK_INTERFACE_LIBRARIES LibCrypto) + +-add_library(LibZ STATIC IMPORTED) +-set_property(TARGET LibZ PROPERTY IMPORTED_LOCATION ${THIRD_PARTY_LIB_DIR}/libz.a) ++add_library(LibZ SHARED IMPORTED) ++set_property(TARGET LibZ PROPERTY IMPORTED_LOCATION ${ZLIB_LIBRARIES}) + +-add_library(LibCurl STATIC IMPORTED) +-set_property(TARGET LibCurl PROPERTY IMPORTED_LOCATION ${THIRD_PARTY_LIB_DIR}/libcurl.a) ++add_library(LibCurl SHARED IMPORTED) ++set_property(TARGET LibCurl PROPERTY IMPORTED_LOCATION ${CURL_LIBRARIES}) + set_property(TARGET LibCurl PROPERTY IMPORTED_LINK_INTERFACE_LIBRARIES ${CMAKE_THREAD_LIBS_INIT} ${LIBRT_LIBRARIES} ${LIBDL_LIBRARIES} LibSsl LibZ) + + add_library(LibProto STATIC IMPORTED) +diff --git a/aws/utils/backtrace/bsd_backtrace.cc b/aws/utils/backtrace/bsd_backtrace.cc +index fb5dbe3..cd6c5fe 100644 +--- a/aws/utils/backtrace/bsd_backtrace.cc ++++ b/aws/utils/backtrace/bsd_backtrace.cc +@@ -15,10 +15,10 @@ + + #include "backtrace.h" + #include +-#include + #include + + #ifdef BSD_BACKTRACE ++#include + namespace aws { + namespace utils { + namespace backtrace { +diff --git a/aws/utils/backtrace/gcc_backtrace.cc b/aws/utils/backtrace/gcc_backtrace.cc +index 446ede9..32a866d 100644 +--- a/aws/utils/backtrace/gcc_backtrace.cc ++++ b/aws/utils/backtrace/gcc_backtrace.cc +@@ -15,7 +15,6 @@ + + #include "backtrace.h" + #include +-#include + #include + #include + #include +@@ -23,7 +22,7 @@ + #include + + #ifdef LIB_BACKTRACE +- ++#include + #include + + namespace { +diff --git a/aws/utils/backtrace/null_backtrace.cc b/aws/utils/backtrace/null_backtrace.cc +index 69d57f9..d443eae 100644 +--- a/aws/utils/backtrace/null_backtrace.cc ++++ b/aws/utils/backtrace/null_backtrace.cc +@@ -15,10 +15,9 @@ + + #include "backtrace.h" + #include +-#include + #include + +-#ifdef NULL_STACKTRACE ++#ifdef NULL_BACKTRACE + + namespace aws { + namespace utils { +@@ -36,4 +35,4 @@ void stack_trace_for_signal(int skip, bool /*signaled*/) { + } + } + +-#endif // NULL_STACKTRACE ++#endif // NULL_BACKTRACE +diff --git a/aws/utils/signal_handler.cc b/aws/utils/signal_handler.cc +index b58ab0e..f483c77 100644 +--- a/aws/utils/signal_handler.cc ++++ b/aws/utils/signal_handler.cc +@@ -19,7 +19,6 @@ + #include "backtrace/backtrace.h" + + #include +-#include + #include + #include + #include +-- +2.47.1 + diff --git a/docker/pulsar-all/Dockerfile b/docker/pulsar-all/Dockerfile index 81ad74b65000f..59ab86e45569e 100644 --- a/docker/pulsar-all/Dockerfile +++ b/docker/pulsar-all/Dockerfile @@ -17,16 +17,42 @@ # under the License. # +# global arguments (only to be used in FROM clauses) ARG PULSAR_IMAGE -FROM busybox as pulsar-all +ARG PULSAR_IO_KINESIS_KPL_IMAGE -ARG PULSAR_IO_DIR -ARG PULSAR_OFFLOADER_TARBALL +FROM busybox AS pulsar-extensions +ARG PULSAR_IO_DIR ADD ${PULSAR_IO_DIR} /connectors +ARG PULSAR_OFFLOADER_TARBALL ADD ${PULSAR_OFFLOADER_TARBALL} / RUN mv /apache-pulsar-offloaders-*/offloaders /offloaders -FROM $PULSAR_IMAGE -COPY --from=pulsar-all /connectors /pulsar/connectors -COPY --from=pulsar-all /offloaders /pulsar/offloaders +FROM ${PULSAR_IO_KINESIS_KPL_IMAGE} AS pulsar-io-kinesis-sink-kinesis_producer + +FROM ${PULSAR_IMAGE} +COPY --from=pulsar-extensions /connectors /pulsar/connectors +COPY --from=pulsar-extensions /offloaders /pulsar/offloaders +# Copy the kinesis_producer native executable compiled for Alpine musl to the pulsar-all image +# This is required to support the Pulsar IO Kinesis sink connector +COPY --from=pulsar-io-kinesis-sink-kinesis_producer /opt/amazon-kinesis-producer/bin/kinesis_producer /opt/amazon-kinesis-producer/bin/.os_info /opt/amazon-kinesis-producer/bin/.build_time /opt/amazon-kinesis-producer/bin/.revision /opt/amazon-kinesis-producer/bin/.system_info /opt/amazon-kinesis-producer/bin/.version /opt/amazon-kinesis-producer/bin/ +# Set the environment variable to point to the kinesis_producer native executable +ENV PULSAR_IO_KINESIS_KPL_PATH=/opt/amazon-kinesis-producer/bin/kinesis_producer +# Install the required dependencies for the kinesis_producer native executable +USER 0 +RUN apk update && apk add --no-cache \ + brotli-libs \ + c-ares \ + libcrypto3 \ + libcurl \ + libgcc \ + libidn2 \ + libpsl \ + libssl3 \ + libunistring \ + nghttp2-libs \ + zlib \ + zstd-libs \ + libuuid +USER 10000 \ No newline at end of file diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 1f365109c7983..5281e2ab38db2 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -80,6 +80,10 @@ docker + + + apachepulsar/pulsar-io-kinesis-sink-kinesis_producer:0.15.12 + ${project.groupId} @@ -161,6 +165,7 @@ target/apache-pulsar-io-connectors-${project.version}-bin target/pulsar-offloader-distribution-${project.version}-bin.tar.gz ${docker.organization}/${docker.image}:${project.version}-${git.commit.id.abbrev} + ${PULSAR_IO_KINESIS_KPL_IMAGE} diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index c8fc3a4d13c8f..6f1f41755c919 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -75,7 +75,7 @@ ARG SNAPPY_VERSION RUN apk add git alpine-sdk util-linux cmake autoconf automake libtool openjdk17 maven curl bash tar ENV JAVA_HOME=/usr RUN curl -Ls https://github.com/xerial/snappy-java/archive/refs/tags/v$SNAPPY_VERSION.tar.gz | tar zxf - && cd snappy-java-$SNAPPY_VERSION && make clean-native native -FROM apachepulsar/glibc-base:2.38 as glibc + ## Create final stage from Alpine image ## and add OpenJDK and Python dependencies (for Pulsar functions) @@ -90,6 +90,9 @@ RUN apk add --no-cache \ py3-grpcio \ py3-yaml \ gcompat \ + libgcc \ + libstdc++ \ + libuuid \ ca-certificates \ procps \ curl \ @@ -124,10 +127,6 @@ fastavro>=1.9.2\n\ RUN pip3 install --break-system-packages --no-cache-dir --only-binary grpcio -r /requirements.txt RUN rm /requirements.txt -# Install GLibc compatibility library -COPY --from=glibc /root/packages /root/packages -RUN apk add --allow-untrusted --force-overwrite /root/packages/glibc-*.apk - COPY --from=jvm /opt/jvm /opt/jvm ENV JAVA_HOME=/opt/jvm @@ -144,6 +143,8 @@ WORKDIR /pulsar ENV PATH=$PATH:$JAVA_HOME/bin:/pulsar/bin # Use musl libc library for RocksDB ENV ROCKSDB_MUSL_LIBC=true +# Preload gcompat library for glibc compatibility with Netty native libraries +ENV LD_PRELOAD=/lib/libgcompat.so.0 # The UID must be non-zero. Otherwise, it is arbitrary. No logic should rely on its specific value. ARG DEFAULT_USERNAME=pulsar diff --git a/pom.xml b/pom.xml index 2a91978af8e58..93cd3d5e11f15 100644 --- a/pom.xml +++ b/pom.xml @@ -166,7 +166,7 @@ flexible messaging model and an intuitive client API. 0.10.2 1.6.2 10.14.2 - 0.45.0 + 0.45.1 true 0.5.0 1.14.12 @@ -2098,6 +2098,7 @@ flexible messaging model and an intuitive client API. **/*.so **/*.so.* **/*.dylib + **/*.patch src/test/resources/*.txt diff --git a/pulsar-io/kinesis/pom.xml b/pulsar-io/kinesis/pom.xml index b472289a472d3..64edc9670a774 100644 --- a/pulsar-io/kinesis/pom.xml +++ b/pulsar-io/kinesis/pom.xml @@ -32,7 +32,7 @@ 2.2.8 - 0.14.13 + 0.15.12 0.13.0 1.9.0 2.3.0 diff --git a/pulsar-io/kinesis/src/main/java/org/apache/pulsar/io/kinesis/KinesisSink.java b/pulsar-io/kinesis/src/main/java/org/apache/pulsar/io/kinesis/KinesisSink.java index d8e4e4bab85e5..1db63c90b2776 100644 --- a/pulsar-io/kinesis/src/main/java/org/apache/pulsar/io/kinesis/KinesisSink.java +++ b/pulsar-io/kinesis/src/main/java/org/apache/pulsar/io/kinesis/KinesisSink.java @@ -88,7 +88,6 @@ configClass = KinesisSinkConfig.class ) public class KinesisSink extends AbstractAwsConnector implements Sink { - private static final Logger LOG = LoggerFactory.getLogger(KinesisSink.class); private KinesisProducer kinesisProducer; @@ -166,6 +165,12 @@ public void open(Map config, SinkContext sinkContext) { if (kinesisSinkConfig.getAwsEndpointPort() != null) { kinesisConfig.setKinesisPort(kinesisSinkConfig.getAwsEndpointPort()); } + if (kinesisSinkConfig.getAwsStsEndpoint() != null) { + kinesisConfig.setStsEndpoint(kinesisSinkConfig.getAwsStsEndpoint()); + } + if (kinesisSinkConfig.getAwsStsPort() != null) { + kinesisConfig.setStsPort(kinesisSinkConfig.getAwsStsPort()); + } kinesisConfig.setRegion(kinesisSinkConfig.getAwsRegion()); kinesisConfig.setThreadingModel(ThreadingModel.POOLED); kinesisConfig.setThreadPoolSize(4); @@ -179,6 +184,7 @@ public void open(Map config, SinkContext sinkContext) { kinesisSinkConfig.getAwsCredentialPluginParam()) .getCredentialProvider(); kinesisConfig.setCredentialsProvider(credentialsProvider); + kinesisConfig.setNativeExecutable(StringUtils.trimToEmpty(kinesisSinkConfig.getNativeExecutable())); this.streamName = kinesisSinkConfig.getAwsKinesisStreamName(); this.kinesisProducer = new KinesisProducer(kinesisConfig); diff --git a/pulsar-io/kinesis/src/main/java/org/apache/pulsar/io/kinesis/KinesisSinkConfig.java b/pulsar-io/kinesis/src/main/java/org/apache/pulsar/io/kinesis/KinesisSinkConfig.java index f81fd32134be2..a3b87d4886a6a 100644 --- a/pulsar-io/kinesis/src/main/java/org/apache/pulsar/io/kinesis/KinesisSinkConfig.java +++ b/pulsar-io/kinesis/src/main/java/org/apache/pulsar/io/kinesis/KinesisSinkConfig.java @@ -104,6 +104,16 @@ public class KinesisSinkConfig extends BaseKinesisConfig implements Serializable help = "The maximum delay(in milliseconds) between retries.") private long retryMaxDelayInMillis = 60000; + @FieldDoc( + required = false, + defaultValue = "", + help = "Path to the native Amazon Kinesis Producer Library (KPL) binary.\n" + + "Only use this setting if you want to use a custom build of the native code.\n" + + "This setting can also be set with the environment variable `PULSAR_IO_KINESIS_KPL_PATH`.\n" + + "If not set, the Kinesis sink will use the built-in native executable." + ) + private String nativeExecutable = System.getenv("PULSAR_IO_KINESIS_KPL_PATH"); + public static KinesisSinkConfig load(Map config, SinkContext sinkContext) { KinesisSinkConfig kinesisSinkConfig = IOConfigUtils.loadWithSecrets(config, KinesisSinkConfig.class, sinkContext); checkArgument(isNotBlank(kinesisSinkConfig.getAwsRegion()) @@ -149,4 +159,17 @@ public enum MessageFormat { FULL_MESSAGE_IN_JSON_EXPAND_VALUE } + @FieldDoc( + required = false, + defaultValue = "", + help = "Custom AWS STS endpoint" + ) + private String awsStsEndpoint = ""; + + @FieldDoc( + required = false, + defaultValue = "", + help = "Custom AWS STS port to connect to" + ) + private Integer awsStsPort; } diff --git a/pulsar-io/kinesis/src/test/java/org/apache/pulsar/io/kinesis/KinesisSinkTest.java b/pulsar-io/kinesis/src/test/java/org/apache/pulsar/io/kinesis/KinesisSinkTest.java index 8a5d5be0ca40c..c682776a14d2b 100644 --- a/pulsar-io/kinesis/src/test/java/org/apache/pulsar/io/kinesis/KinesisSinkTest.java +++ b/pulsar-io/kinesis/src/test/java/org/apache/pulsar/io/kinesis/KinesisSinkTest.java @@ -18,6 +18,16 @@ */ package org.apache.pulsar.io.kinesis; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import lombok.SneakyThrows; import org.apache.pulsar.client.api.Message; @@ -42,24 +52,14 @@ import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; -import java.net.URI; -import java.nio.charset.StandardCharsets; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; - public class KinesisSinkTest { public static final String STREAM_NAME = "my-stream-1"; - public static LocalStackContainer LOCALSTACK_CONTAINER = new LocalStackContainer(DockerImageName.parse("localstack/localstack:1.0.4")) - .withServices(LocalStackContainer.Service.KINESIS); + public static LocalStackContainer LOCALSTACK_CONTAINER = + new LocalStackContainer(DockerImageName.parse("localstack/localstack:4.0.3")) + .withServices(LocalStackContainer.Service.KINESIS, LocalStackContainer.Service.STS) + .withEnv("KINESIS_PROVIDER", "kinesalite"); @BeforeClass(alwaysRun = true) public void beforeClass() throws Exception { @@ -123,10 +123,13 @@ public Optional> getMessage() { } private Map createConfig() { - final URI endpointOverride = LOCALSTACK_CONTAINER.getEndpointOverride(LocalStackContainer.Service.KINESIS); + final URI kinesisEndpointOverride = LOCALSTACK_CONTAINER.getEndpointOverride(LocalStackContainer.Service.KINESIS); Map map = new HashMap<>(); - map.put("awsEndpoint", endpointOverride.getHost()); - map.put("awsEndpointPort", endpointOverride.getPort()); + map.put("awsEndpoint", kinesisEndpointOverride.getHost()); + map.put("awsEndpointPort", kinesisEndpointOverride.getPort()); + final URI stsEndpointOverride = LOCALSTACK_CONTAINER.getEndpointOverride(LocalStackContainer.Service.STS); + map.put("awsStsEndpoint", stsEndpointOverride.getHost()); + map.put("awsStsPort", stsEndpointOverride.getPort()); map.put("skipCertificateValidation", true); map.put("awsKinesisStreamName", STREAM_NAME); map.put("awsRegion", "us-east-1"); diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile index 0645dd2e78aab..3efc481721464 100644 --- a/tests/docker-images/latest-version-image/Dockerfile +++ b/tests/docker-images/latest-version-image/Dockerfile @@ -17,10 +17,9 @@ # under the License. # -# build go lang examples first in a separate layer ARG PULSAR_ALL_IMAGE -ARG PULSAR_IMAGE +# build go lang examples first in a separate layer FROM golang:1.21-alpine as pulsar-function-go COPY target/pulsar-function-go/ /go/src/github.com/apache/pulsar/pulsar-function-go @@ -28,13 +27,10 @@ RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go && go install ./... RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/pf && go install RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/examples && go install ./... -# Reference pulsar-all to copy connectors from there -FROM $PULSAR_ALL_IMAGE as pulsar-all - ######################################## ###### Main image build ######################################## -FROM $PULSAR_IMAGE +FROM $PULSAR_ALL_IMAGE # Switch to run as the root user to simplify building container and then running # supervisord. Each of the pulsar components are spawned by supervisord and their @@ -78,20 +74,6 @@ COPY target/certificate-authority /pulsar/certificate-authority/ # copy broker plugins COPY target/plugins/ /pulsar/examples/ -# Include all offloaders -COPY --from=pulsar-all /pulsar/offloaders /pulsar/offloaders - -# Include only the connectors needed by integration tests -COPY --from=pulsar-all /pulsar/connectors/pulsar-io-cassandra-*.nar /pulsar/connectors/ -COPY --from=pulsar-all /pulsar/connectors/pulsar-io-debezium-*.nar /pulsar/connectors/ -COPY --from=pulsar-all /pulsar/connectors/pulsar-io-elastic-*.nar /pulsar/connectors/ -COPY --from=pulsar-all /pulsar/connectors/pulsar-io-hdfs*.nar /pulsar/connectors/ -COPY --from=pulsar-all /pulsar/connectors/pulsar-io-jdbc-postgres-*.nar /pulsar/connectors/ -COPY --from=pulsar-all /pulsar/connectors/pulsar-io-kafka-*.nar /pulsar/connectors/ -COPY --from=pulsar-all /pulsar/connectors/pulsar-io-rabbitmq-*.nar /pulsar/connectors/ -COPY --from=pulsar-all /pulsar/connectors/pulsar-io-batch-data-generator-*.nar /pulsar/connectors/ -COPY --from=pulsar-all /pulsar/connectors/pulsar-io-kinesis-*.nar /pulsar/connectors/ - # download Oracle JDBC driver for Oracle Debezium Connector tests RUN mkdir -p META-INF/bundled-dependencies RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/ojdbc8/19.3.0.0/ojdbc8-19.3.0.0.jar -o ojdbc8-19.3.0.0.jar @@ -110,4 +92,4 @@ RUN jar uf connectors/pulsar-io-debezium-oracle-*.nar META-INF/bundled-dependenc RUN mkdir -p pulsar RUN chmod g+rwx pulsar -CMD bash +CMD bash \ No newline at end of file diff --git a/tests/docker-images/latest-version-image/pom.xml b/tests/docker-images/latest-version-image/pom.xml index 14984d809ac8d..e767d2a2f37e9 100644 --- a/tests/docker-images/latest-version-image/pom.xml +++ b/tests/docker-images/latest-version-image/pom.xml @@ -156,7 +156,6 @@ ${project.basedir} - ${docker.organization}/${docker.image}:${project.version}-${git.commit.id.abbrev} ${docker.organization}/${docker.image}-all:${project.version}-${git.commit.id.abbrev} diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/KinesisSinkTester.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/KinesisSinkTester.java index 6ca9c7b108373..83cb0088cd76b 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/KinesisSinkTester.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/KinesisSinkTester.java @@ -79,6 +79,11 @@ public KinesisSinkTester(boolean withSchema) { sinkConfig.put("awsKinesisStreamName", STREAM_NAME); sinkConfig.put("awsRegion", "us-east-1"); sinkConfig.put("awsCredentialPluginParam", "{\"accessKey\":\"access\",\"secretKey\":\"secret\"}"); + sinkConfig.put("awsEndpoint", NAME); + sinkConfig.put("awsEndpointPort", LOCALSTACK_SERVICE_PORT); + sinkConfig.put("awsStsEndpoint", NAME); + sinkConfig.put("awsStsPort", LOCALSTACK_SERVICE_PORT); + sinkConfig.put("skipCertificateValidation", true); if (withSchema) { sinkConfig.put("messageFormat", "FULL_MESSAGE_IN_JSON_EXPAND_VALUE"); } @@ -100,9 +105,6 @@ public Schema getInputTopicSchema() { public void prepareSink() throws Exception { final LocalStackContainer localStackContainer = getServiceContainer(); final URI endpointOverride = localStackContainer.getEndpointOverride(LocalStackContainer.Service.KINESIS); - sinkConfig.put("awsEndpoint", NAME); - sinkConfig.put("awsEndpointPort", LOCALSTACK_SERVICE_PORT); - sinkConfig.put("skipCertificateValidation", true); client = KinesisAsyncClient.builder().credentialsProvider(() -> AwsBasicCredentials.create( "access", "secret")) @@ -128,8 +130,9 @@ public void stopServiceContainer() { @Override protected LocalStackContainer createSinkService(PulsarCluster cluster) { - return new LocalStackContainer(DockerImageName.parse("localstack/localstack:1.0.4")) - .withServices(LocalStackContainer.Service.KINESIS); + return new LocalStackContainer(DockerImageName.parse("localstack/localstack:4.0.3")) + .withServices(LocalStackContainer.Service.KINESIS, LocalStackContainer.Service.STS) + .withEnv("KINESIS_PROVIDER", "kinesalite"); } @Override From 3d505742e05e403144d17aef59c95e2758982a26 Mon Sep 17 00:00:00 2001 From: danpi Date: Sat, 21 Dec 2024 19:20:56 +0800 Subject: [PATCH 217/327] [fix][admin] Fix exception loss in getMessageId method (#23766) Co-authored-by: houbonan --- .../pulsar/broker/admin/impl/PersistentTopicsBase.java | 7 ++++--- .../apache/pulsar/broker/admin/PersistentTopicsTest.java | 4 ++++ 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 1300cd3449c27..4b9fea05cd885 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -2774,9 +2774,10 @@ protected CompletableFuture internalGetMessageById(long ledgerId, long public void readEntryFailed(ManagedLedgerException exception, Object ctx) { if (exception instanceof ManagedLedgerException.LedgerNotExistException) { - throw new RestException(Status.NOT_FOUND, "Message id not found"); + results.completeExceptionally( + new RestException(Status.NOT_FOUND, "Message id not found")); } - throw new RestException(exception); + results.completeExceptionally(new RestException(exception)); } @Override @@ -2784,7 +2785,7 @@ public void readEntryComplete(Entry entry, Object ctx) { try { results.complete(generateResponseWithEntry(entry, (PersistentTopic) topic)); } catch (IOException exception) { - throw new RestException(exception); + results.completeExceptionally(new RestException(exception)); } finally { if (entry != null) { entry.release(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index 302948903442c..258c0183114fd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -1419,6 +1419,10 @@ public void testGetMessageById() throws Exception { Assert.expectThrows(PulsarAdminException.NotFoundException.class, () -> { admin.topics().getMessageById(topicName1, id2.getLedgerId(), id2.getEntryId()); }); + + Assert.expectThrows(PulsarAdminException.ServerSideErrorException.class, () -> { + admin.topics().getMessageById(topicName1, id1.getLedgerId(), id1.getEntryId() + 10); + }); } @Test From 1967a9309586286580ac0f3b75a34e1f70e63f75 Mon Sep 17 00:00:00 2001 From: hrzzzz <64506104+hrzzzz@users.noreply.github.com> Date: Sat, 21 Dec 2024 21:32:32 +0800 Subject: [PATCH 218/327] [fix][broker] System topic should not be migrated during blue-green cluster migration (#23767) Co-authored-by: ruihongzhou --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 4 +++- .../pulsar/broker/service/persistent/PersistentTopic.java | 5 ++++- 2 files changed, 7 insertions(+), 2 deletions(-) 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 6afa1ae32fbcb..c79d839097e68 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 @@ -1888,7 +1888,9 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { } private CompletableFuture checkTopicAlreadyMigrated(TopicName topicName) { - if (ExtensibleLoadManagerImpl.isInternalTopic(topicName.toString())) { + if (ExtensibleLoadManagerImpl.isInternalTopic(topicName.toString()) + || SystemTopicNames.isEventSystemTopic(topicName) + || NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { return CompletableFuture.completedFuture(null); } CompletableFuture result = new CompletableFuture<>(); 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 11220d1c955e3..f6fe1a2f8de4d 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 @@ -2926,7 +2926,10 @@ private boolean hasBacklogs(boolean getPreciseBacklog) { @Override public CompletableFuture checkClusterMigration() { - if (ExtensibleLoadManagerImpl.isInternalTopic(topic)) { + TopicName topicName = TopicName.get(topic); + if (ExtensibleLoadManagerImpl.isInternalTopic(topic) + || isEventSystemTopic(topicName) + || NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { return CompletableFuture.completedFuture(null); } From 14129e352e4bb4e06cbf2c6c581edcdbaf9f6775 Mon Sep 17 00:00:00 2001 From: Anurag reddy K <62467911+AnuragReddy2000@users.noreply.github.com> Date: Tue, 24 Dec 2024 12:11:42 +0530 Subject: [PATCH 219/327] [fix][client] Make DeadLetterPolicy & KeySharedPolicy serializable (#23718) Co-authored-by: anurag.reddy --- .../broker/service/BatchMessageTest.java | 5 +- .../pulsar/client/api/DeadLetterPolicy.java | 4 +- .../pulsar/client/api/KeySharedPolicy.java | 5 +- .../org/apache/pulsar/client/api/Range.java | 4 +- .../impl/conf/ConsumerConfigurationData.java | 4 +- .../impl/conf/ReaderConfigurationData.java | 2 +- .../conf/ConsumerConfigurationDataTest.java | 50 +++++++++++++++++++ 7 files changed, 66 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java index 2fd288239e362..e5f9e43b8bb4a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java @@ -991,8 +991,8 @@ public void testBatchMessageDispatchingAccordingToPermits() throws Exception { int numMsgs = 1000; int batchMessages = 10; - final String topicName = "persistent://prop/ns-abc/testRetrieveSequenceIdSpecify-" + UUID.randomUUID(); - final String subscriptionName = "sub-1"; + final String topicName = "persistent://prop/ns-abc/testBatchMessageDispatchingAccordingToPermits-" + UUID.randomUUID(); + final String subscriptionName = "bmdap-sub-1"; ConsumerImpl consumer1 = (ConsumerImpl) pulsarClient.newConsumer().topic(topicName) .subscriptionName(subscriptionName).receiverQueueSize(10).subscriptionType(SubscriptionType.Shared) @@ -1017,6 +1017,7 @@ public void testBatchMessageDispatchingAccordingToPermits() throws Exception { producer.close(); consumer1.close(); + consumer2.close(); } @Test(dataProvider="testSubTypeAndEnableBatch") diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/DeadLetterPolicy.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/DeadLetterPolicy.java index f0b75ff57dd7b..c2a172666b01d 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/DeadLetterPolicy.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/DeadLetterPolicy.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import java.io.Serializable; import lombok.AllArgsConstructor; import lombok.Builder; import lombok.Data; @@ -36,7 +37,8 @@ @AllArgsConstructor @InterfaceAudience.Public @InterfaceStability.Stable -public class DeadLetterPolicy { +public class DeadLetterPolicy implements Serializable { + private static final long serialVersionUID = 1L; /** * Maximum number of times that a message will be redelivered before being sent to the dead letter queue. diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/KeySharedPolicy.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/KeySharedPolicy.java index f5bc5b846b6db..ccaed04d75da3 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/KeySharedPolicy.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/KeySharedPolicy.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -29,7 +30,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public abstract class KeySharedPolicy { +public abstract class KeySharedPolicy implements Serializable { protected KeySharedMode keySharedMode; @@ -82,6 +83,7 @@ public int getHashRangeTotal() { * for message, the cursor will rewind. */ public static class KeySharedPolicySticky extends KeySharedPolicy { + private static final long serialVersionUID = 1L; protected final List ranges; @@ -129,6 +131,7 @@ public List getRanges() { * Auto split hash range key shared policy. */ public static class KeySharedPolicyAutoSplit extends KeySharedPolicy { + private static final long serialVersionUID = 1L; KeySharedPolicyAutoSplit() { this.keySharedMode = KeySharedMode.AUTO_SPLIT; diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java index 3db225330d0c7..14d9eec0e1b46 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import java.io.Serializable; import java.util.Objects; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; @@ -27,7 +28,8 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class Range implements Comparable { +public class Range implements Comparable, Serializable { + private static final long serialVersionUID = 1L; private final int start; private final int end; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index 6e884ba279116..cd82b54618f22 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -359,7 +359,7 @@ public int getMaxPendingChuckedMessage() { + "When specifying the dead letter policy while not specifying `ackTimeoutMillis`, you can set the" + " ack timeout to 30000 millisecond." ) - private transient DeadLetterPolicy deadLetterPolicy; + private DeadLetterPolicy deadLetterPolicy; private boolean retryEnable = false; @@ -388,7 +388,7 @@ public int getMaxPendingChuckedMessage() { private boolean resetIncludeHead = false; @JsonIgnore - private transient KeySharedPolicy keySharedPolicy; + private KeySharedPolicy keySharedPolicy; private boolean batchIndexAckEnabled = false; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ReaderConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ReaderConfigurationData.java index 73d97f1f33607..cd5aa4c12f5c3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ReaderConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ReaderConfigurationData.java @@ -144,7 +144,7 @@ public class ReaderConfigurationData implements Serializable, Cloneable { ) private boolean resetIncludeHead = false; - private transient List keyHashRanges; + private List keyHashRanges; private boolean poolMessages = false; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationDataTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationDataTest.java index f47f83bcbce32..291583c306746 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationDataTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationDataTest.java @@ -19,7 +19,18 @@ package org.apache.pulsar.client.impl.conf; import static org.assertj.core.api.Assertions.assertThat; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Collections; import java.util.regex.Pattern; + +import lombok.Cleanup; +import org.apache.pulsar.client.api.DeadLetterPolicy; +import org.apache.pulsar.client.api.SubscriptionType; +import org.testng.Assert; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -45,4 +56,43 @@ public void testTopicConsumerConfigurationData(String topicName, int expectedPri assertThat(topicConsumerConfigurationData.getPriorityLevel()).isEqualTo(expectedPriority); } + + @Test + public void testSerializable() throws Exception { + ConsumerConfigurationData consumerConfigurationData = new ConsumerConfigurationData<>(); + consumerConfigurationData.setPriorityLevel(1); + consumerConfigurationData.setSubscriptionName("my-sub"); + consumerConfigurationData.setSubscriptionType(SubscriptionType.Shared); + consumerConfigurationData.setReceiverQueueSize(100); + consumerConfigurationData.setAckTimeoutMillis(1000); + consumerConfigurationData.setTopicNames(Collections.singleton("my-topic")); + + DeadLetterPolicy deadLetterPolicy = DeadLetterPolicy.builder() + .maxRedeliverCount(10) + .retryLetterTopic("retry-topic") + .deadLetterTopic("dead-topic") + .build(); + consumerConfigurationData.setDeadLetterPolicy(deadLetterPolicy); + + @Cleanup + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + @Cleanup + ObjectOutputStream oos = new ObjectOutputStream(bos); + oos.writeObject(consumerConfigurationData); + byte[] serialized = bos.toByteArray(); + + // Deserialize + @Cleanup + ByteArrayInputStream bis = new ByteArrayInputStream(serialized); + @Cleanup + ObjectInputStream ois = new ObjectInputStream(bis); + Object object = ois.readObject(); + + Assert.assertEquals(object.getClass(), ConsumerConfigurationData.class); + Assert.assertEquals(object, consumerConfigurationData); + + DeadLetterPolicy deserialisedDeadLetterPolicy = ((ConsumerConfigurationData) object).getDeadLetterPolicy(); + Assert.assertNotNull(deserialisedDeadLetterPolicy); + Assert.assertEquals(deserialisedDeadLetterPolicy, deadLetterPolicy); + } } From 9699dc25deb5eb701de88dc209e6c42efe58556c Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 25 Dec 2024 14:14:01 +0800 Subject: [PATCH 220/327] [fix][ml] Topic load timeout due to ml data ledger future never finishes (#23772) ### Motivation **Background** There is a mechanism that repeatedly prevents the callback of ML data ledger creation: - Start a scheduled task to check whether the creation will be timeout. - Received a callback - Check whether the future(`@param ctx` of `BK.createAsync`) has been done or not. - If done: it means the creation has timeout before the creation is completed - Otherwise: it is a real callback from BK. **Issue:** But the timeout event will call the same callback as above, then the steps are as follows, which you ca reproduce by the test `testCreateDataLedgerTimeout`: - Start creating a data ledger - Call `BK.createAsync` - Timeout - Mark the future(`@param ctx` of `BK.createAsync`) as completed exceptionally. - Trigger the callback related to ledger creation. - Check whether the future(`@param ctx` of `BK.createAsync`) has been done or not. - If done: do nothing. - Creation is compelled. - Trigger the callback related to ledger creation. - Check whether the future(`@param ctx` of `BK.createAsync`) has been done or not. - If done: do nothing. - Issue: The callback for ledger creation will never be called. ![Screenshot 2024-12-24 at 00 14 38](https://github.com/user-attachments/assets/44ed19d2-7238-45a4-9186-c127f6ed14f7) ![Screenshot 2024-12-24 at 00 14 08](https://github.com/user-attachments/assets/349f39ff-7e98-4a09-9af2-f80082339592) ### Modifications Fix the issue ### Documentation - [ ] `doc` - [ ] `doc-required` - [x] `doc-not-needed` - [ ] `doc-complete` ### Matching PR in forked repository PR in forked repository: x --- .../mledger/impl/ManagedLedgerImpl.java | 3 ++- .../mledger/impl/ManagedLedgerTest.java | 25 +++++++++++++++++++ .../client/PulsarMockBookKeeper.java | 15 ++++++++++- 3 files changed, 41 insertions(+), 2 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 926e7c7f8108e..966aa068f2ff5 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 @@ -574,6 +574,7 @@ public void operationFailed(MetaStoreException e) { executor.execute(() -> { mbean.endDataLedgerCreateOp(); if (rc != BKException.Code.OK) { + log.error("[{}] Error creating ledger rc={} {}", name, rc, BKException.getMessage(rc)); callback.initializeFailed(createManagedLedgerException(rc)); return; } @@ -4141,7 +4142,7 @@ public Clock getClock() { protected boolean checkAndCompleteLedgerOpTask(int rc, LedgerHandle lh, Object ctx) { if (ctx instanceof CompletableFuture) { // ledger-creation is already timed out and callback is already completed so, delete this ledger and return. - if (((CompletableFuture) ctx).complete(lh)) { + if (((CompletableFuture) ctx).complete(lh) || rc == BKException.Code.TimeoutException) { return false; } else { if (rc == BKException.Code.OK) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 67c928c8a5617..04f8eecbe9a3b 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -4236,6 +4236,31 @@ public void testNonDurableCursorCreateForInactiveLedger() throws Exception { assertNotNull(ml.newNonDurableCursor(Position)); } + @Test(timeOut = 60 * 1000) + public void testCreateDataLedgerTimeout() throws Exception { + String mlName = UUID.randomUUID().toString(); + ManagedLedgerFactoryImpl factory = null; + ManagedLedger ml = null; + try { + factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMetadataOperationsTimeoutSeconds(5); + bkc.delay(10 * 1000); + ml = factory.open(mlName, config); + fail("Should get a timeout ex"); + } catch (ManagedLedgerException ex) { + assertTrue(ex.getMessage().contains("timeout")); + } finally { + // cleanup. + if (ml != null) { + ml.delete(); + } + if (factory != null) { + factory.shutdown(); + } + } + } + /*** * When a ML tries to create a ledger, it will create a delay task to check if the ledger create request is timeout. * But we should guarantee that the delay task should be canceled after the ledger create request responded. diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java index 11ec2dec938a8..1e979206e16d3 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.common.collect.Lists; +import io.netty.util.concurrent.DefaultThreadFactory; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -35,6 +36,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -70,6 +72,7 @@ public class PulsarMockBookKeeper extends BookKeeper { final OrderedExecutor orderedExecutor; final ExecutorService executor; + final ScheduledExecutorService scheduler; @Override public ClientConfiguration getConf() { @@ -97,6 +100,7 @@ public static Collection getMockEnsemble() { public PulsarMockBookKeeper(OrderedExecutor orderedExecutor) throws Exception { this.orderedExecutor = orderedExecutor; this.executor = orderedExecutor.chooseThread(); + scheduler = Executors.newScheduledThreadPool(1, new DefaultThreadFactory("mock-bk-scheduler")); } @Override @@ -290,7 +294,7 @@ public void shutdown() { for (PulsarMockLedgerHandle ledger : ledgers.values()) { ledger.entries.clear(); } - + scheduler.shutdown(); ledgers.clear(); } @@ -331,6 +335,15 @@ synchronized CompletableFuture getProgrammedFailure() { return failures.isEmpty() ? defaultResponse : failures.remove(0); } + public void delay(long millis) { + CompletableFuture delayFuture = new CompletableFuture<>(); + scheduler.schedule(() -> { + delayFuture.complete(null); + }, millis, TimeUnit.MILLISECONDS); + failures.add(delayFuture); + } + + public void failNow(int rc) { failAfter(0, rc); } From 369c352664a68acbb9bb3584b433498440a41034 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 25 Dec 2024 14:37:15 +0800 Subject: [PATCH 221/327] [improve][ml] Optimize BlobStoreManagedLedgerOffloader.getOffloadPolicies (#23776) --- .../jcloud/impl/BlobStoreManagedLedgerOffloader.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java index b4ed940c9cdca..d6d298159ea22 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java @@ -98,6 +98,7 @@ public class BlobStoreManagedLedgerOffloader implements LedgerOffloader { private final OrderedScheduler scheduler; private final TieredStorageConfiguration config; + private final OffloadPolicies policies; private final Location writeLocation; // metadata to be stored as part of the offloaded ledger metadata @@ -105,8 +106,8 @@ public class BlobStoreManagedLedgerOffloader implements LedgerOffloader { private final ConcurrentMap blobStores = new ConcurrentHashMap<>(); private OffloadSegmentInfoImpl segmentInfo; - private AtomicLong bufferLength = new AtomicLong(0); - private AtomicLong segmentLength = new AtomicLong(0); + private final AtomicLong bufferLength = new AtomicLong(0); + private final AtomicLong segmentLength = new AtomicLong(0); private final long maxBufferLength; private final OffsetsCache entryOffsetsCache; private final ConcurrentLinkedQueue offloadBuffer = new ConcurrentLinkedQueue<>(); @@ -138,6 +139,9 @@ public static BlobStoreManagedLedgerOffloader create(TieredStorageConfiguration this.scheduler = scheduler; this.userMetadata = userMetadata; this.config = config; + Properties properties = new Properties(); + properties.putAll(config.getConfigProperties()); + this.policies = OffloadPoliciesImpl.create(properties); this.streamingBlockSize = config.getMinBlockSizeInBytes(); this.maxSegmentCloseTime = Duration.ofSeconds(config.getMaxSegmentTimeInSecond()); this.maxSegmentLength = config.getMaxSegmentSizeInBytes(); @@ -658,9 +662,7 @@ public CompletableFuture deleteOffloaded(UUID uid, Map off @Override public OffloadPolicies getOffloadPolicies() { - Properties properties = new Properties(); - properties.putAll(config.getConfigProperties()); - return OffloadPoliciesImpl.create(properties); + return this.policies; } @Override From 6e3eaf5150907ecb12bfec3022c26467c16e4721 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 27 Dec 2024 19:19:03 +0800 Subject: [PATCH 222/327] [fix][broker] Fix enableReplicatedSubscriptions (#23781) Signed-off-by: Zixuan Liu --- .../service/persistent/PersistentTopic.java | 6 +- ...ReplicatedSubscriptionsIsDisabledTest.java | 83 +++++++++++++++++++ 2 files changed, 87 insertions(+), 2 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/EnableReplicatedSubscriptionsIsDisabledTest.java 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 f6fe1a2f8de4d..56aeb9b4a5eb3 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 @@ -911,9 +911,11 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St } return brokerService.checkTopicNsOwnership(getName()).thenCompose(__ -> { - if (replicatedSubscriptionStateArg != null && replicatedSubscriptionStateArg + Boolean replicatedSubscriptionState = replicatedSubscriptionStateArg; + if (replicatedSubscriptionState != null && replicatedSubscriptionState && !brokerService.pulsar().getConfiguration().isEnableReplicatedSubscriptions()) { log.warn("[{}] Replicated Subscription is disabled by broker.", getName()); + replicatedSubscriptionState = false; } if (subType == SubType.Key_Shared @@ -982,7 +984,7 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St CompletableFuture subscriptionFuture = isDurable ? getDurableSubscription(subscriptionName, initialPosition, startMessageRollbackDurationSec, - replicatedSubscriptionStateArg, subscriptionProperties) + replicatedSubscriptionState, subscriptionProperties) : getNonDurableSubscription(subscriptionName, startMessageId, initialPosition, startMessageRollbackDurationSec, readCompacted, subscriptionProperties); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/EnableReplicatedSubscriptionsIsDisabledTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/EnableReplicatedSubscriptionsIsDisabledTest.java new file mode 100644 index 0000000000000..d002261cee4a3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/EnableReplicatedSubscriptionsIsDisabledTest.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.common.naming.TopicName; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Test(groups = "broker-api") +public class EnableReplicatedSubscriptionsIsDisabledTest extends ProducerConsumerBase { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setEnableReplicatedSubscriptions(false); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testReplicateSubscriptionStateIsEnabled() throws Exception { + String topicName = TopicName.get("my-property/my-ns/testReplicateSubscriptionStateIsEnabled").toString(); + String subName = "my-subscription"; + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) + .topic(topicName) + .subscriptionName(subName) + .replicateSubscriptionState(true) + .subscribe(); + CompletableFuture> topicIfExists = pulsar.getBrokerService().getTopicIfExists(topicName); + assertThat(topicIfExists) + .succeedsWithin(3, TimeUnit.SECONDS) + .matches(optionalTopic -> { + assertTrue(optionalTopic.isPresent()); + Topic topicRef = optionalTopic.get(); + Subscription subscription = topicRef.getSubscription(subName); + assertNotNull(subscription); + assertTrue(subscription instanceof PersistentSubscription); + PersistentSubscription persistentSubscription = (PersistentSubscription) subscription; + assertEquals(persistentSubscription.getReplicatedControlled(), Boolean.FALSE); + return true; + }); + } +} From 5a3a1f169a7f90181bd5c213c8e9f479bc74f0f2 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 30 Dec 2024 10:40:37 +0800 Subject: [PATCH 223/327] [improve][log] Print ZK path if write to ZK fails due to data being too large to persist (#23652) --- .../apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 6 +++--- .../org/apache/pulsar/metadata/impl/ZKMetadataStore.java | 8 +++++++- .../apache/pulsar/metadata/impl/batching/MetadataOp.java | 2 ++ 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 022cecf8d57b5..0cd9fc0d54cc8 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -2247,6 +2247,8 @@ public void operationFailed(ManagedLedgerException exception) { if (State.NoLedger.equals(STATE_UPDATER.get(this))) { if (ledger.isNoMessagesAfterPos(mdEntry.newPosition)) { + log.error("[{}][{}] Metadata ledger creation failed, try to persist the position in the metadata" + + " store.", ledger.getName(), name); persistPositionToMetaStore(mdEntry, cb); } else { cb.operationFailed(new ManagedLedgerException("Switch new cursor ledger failed")); @@ -2969,9 +2971,7 @@ public void operationComplete() { @Override public void operationFailed(ManagedLedgerException exception) { - log.error("[{}][{}] Metadata ledger creation failed {}, try to persist the position in the metadata" - + " store.", ledger.getName(), name, exception); - + log.error("[{}][{}] Metadata ledger creation failed {}", ledger.getName(), name, exception); synchronized (pendingMarkDeleteOps) { // At this point we don't have a ledger ready STATE_UPDATER.set(ManagedCursorImpl.this, State.NoLedger); 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 603a4503dc8bb..4c24aa5938b93 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 @@ -32,6 +32,7 @@ import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataStore; @@ -201,10 +202,15 @@ protected void batchOperation(List ops) { Collectors.groupingBy(MetadataOp::getType, Collectors.summingInt(op -> 1))) .entrySet().stream().map(e -> e.getValue() + " " + e.getKey().name() + " entries") .collect(Collectors.joining(", ")); + List opsForLog = ops.stream() + .filter(item -> item.size() > 256 * 1024) + .map(op -> Pair.of(op.getPath(), op.size())) + .collect(Collectors.toList()); Long totalSize = ops.stream().collect(Collectors.summingLong(MetadataOp::size)); log.warn("Connection loss while executing batch operation of {} " + "of total data size of {}. " - + "Retrying individual operations one-by-one.", countsByType, totalSize); + + "Retrying individual operations one-by-one. ops whose size > 256KB: {}", + countsByType, totalSize, opsForLog); // Retry with the individual operations executor.schedule(() -> { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/MetadataOp.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/MetadataOp.java index abf60f7b7245c..06ff425372b58 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/MetadataOp.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/MetadataOp.java @@ -51,4 +51,6 @@ default OpGetChildren asGetChildren() { default OpPut asPut() { return (OpPut) this; } + + String getPath(); } From 86f8a84409116e5e42a9463189c0dedaf5cd291a Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Mon, 30 Dec 2024 23:57:27 +0800 Subject: [PATCH 224/327] [fix][broker] topic policy deadlock block metadata thread. (#23786) --- .../SystemTopicBasedTopicPoliciesService.java | 55 +++++++++---------- 1 file changed, 27 insertions(+), 28 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index cc3938491e637..5488d5563f607 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -37,6 +37,9 @@ import javax.annotation.Nonnull; import org.apache.commons.lang3.concurrent.ConcurrentInitializer; import org.apache.commons.lang3.concurrent.LazyInitializer; +import org.apache.commons.lang3.mutable.Mutable; +import org.apache.commons.lang3.mutable.MutableObject; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceBundleOwnershipListener; @@ -267,37 +270,33 @@ public CompletableFuture> getTopicPoliciesAsync(TopicNam return CompletableFuture.completedFuture(Optional.empty()); } final CompletableFuture preparedFuture = prepareInitPoliciesCacheAsync(topicName.getNamespaceObject()); - final var resultFuture = new CompletableFuture>(); - preparedFuture.thenAccept(inserted -> policyCacheInitMap.compute(namespace, (___, existingFuture) -> { - if (!inserted || existingFuture != null) { - final var partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); - final var policies = Optional.ofNullable(switch (type) { - case DEFAULT -> Optional.ofNullable(policiesCache.get(partitionedTopicName)) - .orElseGet(() -> globalPoliciesCache.get(partitionedTopicName)); - case GLOBAL_ONLY -> globalPoliciesCache.get(partitionedTopicName); - case LOCAL_ONLY -> policiesCache.get(partitionedTopicName); - }); - resultFuture.complete(policies); - } else { - CompletableFuture.runAsync(() -> { - log.info("The future of {} has been removed from cache, retry getTopicPolicies again", namespace); - // Call it in another thread to avoid recursive update because getTopicPoliciesAsync() could call - // policyCacheInitMap.computeIfAbsent() - getTopicPoliciesAsync(topicName, type).whenComplete((result, e) -> { - if (e == null) { - resultFuture.complete(result); - } else { - resultFuture.completeExceptionally(e); - } + // switch thread to avoid potential metadata thread cost and recursive deadlock + return preparedFuture.thenComposeAsync(inserted -> { + // initialized : policies + final Mutable>> policiesFutureHolder = new MutableObject<>(); + // NOTICE: avoid using any callback with lock scope to avoid deadlock + policyCacheInitMap.compute(namespace, (___, existingFuture) -> { + if (!inserted || existingFuture != null) { + final var partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); + final var policies = Optional.ofNullable(switch (type) { + case DEFAULT -> Optional.ofNullable(policiesCache.get(partitionedTopicName)) + .orElseGet(() -> globalPoliciesCache.get(partitionedTopicName)); + case GLOBAL_ONLY -> globalPoliciesCache.get(partitionedTopicName); + case LOCAL_ONLY -> policiesCache.get(partitionedTopicName); }); - }); + policiesFutureHolder.setValue(Pair.of(true, policies)); + } else { + policiesFutureHolder.setValue(Pair.of(false, null)); + } + return existingFuture; + }); + final var p = policiesFutureHolder.getValue(); + if (!p.getLeft()) { + log.info("The future of {} has been removed from cache, retry getTopicPolicies again", namespace); + return getTopicPoliciesAsync(topicName, type); } - return existingFuture; - })).exceptionally(e -> { - resultFuture.completeExceptionally(e); - return null; + return CompletableFuture.completedFuture(p.getRight()); }); - return resultFuture; } public void addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { From fdfe00a3c270cc7bf630224957d9fa86958d9e6e Mon Sep 17 00:00:00 2001 From: Yike Xiao Date: Tue, 31 Dec 2024 02:25:09 +0800 Subject: [PATCH 225/327] [fix][client] Cannot access message data inside ProducerInterceptor#onSendAcknowledgement (#23791) --- .../pulsar/client/api/InterceptorsTest.java | 44 +++++++++++++++++++ .../pulsar/client/impl/ProducerImpl.java | 28 ++++++------ 2 files changed, 59 insertions(+), 13 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java index f71cdc551411b..68d082adbb6b7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java @@ -21,10 +21,12 @@ import com.google.common.collect.Sets; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -216,6 +218,48 @@ public void onSendAcknowledgement(Producer producer, Message mes producer.close(); } + @Test + public void testProducerInterceptorAccessMessageData() throws PulsarClientException { + List messageDataInBeforeSend = Collections.synchronizedList(new ArrayList<>()); + List messageDataOnSendAcknowledgement = Collections.synchronizedList(new ArrayList<>()); + ProducerInterceptor interceptor = new ProducerInterceptor<>() { + @Override + public void close() { + } + + @Override + public Message beforeSend(Producer producer, Message message) { + messageDataInBeforeSend.add(new String(message.getData())); + return message; + } + + @Override + public void onSendAcknowledgement(Producer producer, Message message, MessageId msgId, + Throwable exception) { + messageDataOnSendAcknowledgement.add(new String(message.getData())); + } + }; + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic("persistent://my-property/my-ns/my-topic") + .intercept(interceptor) + .create(); + + final String messageValue = UUID.randomUUID().toString(); + try { + producer.newMessage().value(messageValue).send(); + } catch (Exception ignore) { + } + Assert.assertEquals(messageDataInBeforeSend.size(), 1, + "Message data should be available in beforeSend"); + Assert.assertEquals(messageDataInBeforeSend.get(0), messageValue, + "Message data should be available in beforeSend"); + Assert.assertEquals(messageDataOnSendAcknowledgement.size(), 1, + "Message data should be available in onSendAcknowledgement"); + Assert.assertEquals(messageDataOnSendAcknowledgement.get(0), messageValue, + "Message data should be available in onSendAcknowledgement"); + } + @Test public void testConsumerInterceptorWithErrors() throws PulsarClientException { ConsumerInterceptor interceptor = new ConsumerInterceptor() { 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 54d337925dcf9..64b706cc5faa9 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 @@ -436,20 +436,22 @@ private void onSendComplete(Throwable e, SendCallback sendCallback, MessageImpl< if (payload == null) { log.error("[{}] [{}] Payload is null when calling onSendComplete, which is not expected.", topic, producerName); - } else { - ReferenceCountUtil.safeRelease(payload); } - if (e != null) { - latencyHistogram.recordFailure(latencyNanos); - stats.incrementSendFailed(); - onSendAcknowledgement(msg, null, e); - sendCallback.getFuture().completeExceptionally(e); - } else { - latencyHistogram.recordSuccess(latencyNanos); - publishedBytesCounter.add(msgSize); - stats.incrementNumAcksReceived(latencyNanos); - onSendAcknowledgement(msg, msg.getMessageId(), null); - sendCallback.getFuture().complete(msg.getMessageId()); + try { + if (e != null) { + latencyHistogram.recordFailure(latencyNanos); + stats.incrementSendFailed(); + onSendAcknowledgement(msg, null, e); + sendCallback.getFuture().completeExceptionally(e); + } else { + latencyHistogram.recordSuccess(latencyNanos); + publishedBytesCounter.add(msgSize); + stats.incrementNumAcksReceived(latencyNanos); + onSendAcknowledgement(msg, msg.getMessageId(), null); + sendCallback.getFuture().complete(msg.getMessageId()); + } + } finally { + ReferenceCountUtil.safeRelease(payload); } } From 149b9f770a44e693729e850689aa215d4d5af25c Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 31 Dec 2024 10:15:41 +0800 Subject: [PATCH 226/327] [fix][broker] Fix pulsar.replicated.subscription checks (#23782) Signed-off-by: Zixuan Liu --- .../persistent/PersistentSubscription.java | 10 +++-- .../service/persistent/PersistentTopic.java | 9 +++-- .../PersistentSubscriptionTest.java | 37 +++++++++++++++++++ 3 files changed, 50 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 8cebbd52695a8..b5a1a9db5deb1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -118,7 +118,7 @@ public class PersistentSubscription extends AbstractSubscription { // for connected subscriptions, message expiry will be checked if the backlog is greater than this threshold private static final int MINIMUM_BACKLOG_FOR_EXPIRY_CHECK = 1000; - private static final String REPLICATED_SUBSCRIPTION_PROPERTY = "pulsar.replicated.subscription"; + protected static final String REPLICATED_SUBSCRIPTION_PROPERTY = "pulsar.replicated.subscription"; // Map of properties that is used to mark this subscription as "replicated". // Since this is the only field at this point, we can just keep a static @@ -140,8 +140,12 @@ static Map getBaseCursorProperties(Boolean isReplicated) { NON_REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES; } - static boolean isCursorFromReplicatedSubscription(ManagedCursor cursor) { - return cursor.getProperties().containsKey(REPLICATED_SUBSCRIPTION_PROPERTY); + static Optional getReplicatedSubscriptionConfiguration(ManagedCursor cursor) { + Long v = cursor.getProperties().get(REPLICATED_SUBSCRIPTION_PROPERTY); + if (v == null || (v < 0L || v > 1L)) { + return Optional.empty(); + } + return Optional.of(v == 1L); } public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, 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 56aeb9b4a5eb3..08a481b7051c6 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 @@ -512,9 +512,12 @@ private void createPersistentSubscriptions() { // ignore it for now and let the message dedup logic to take care of it } else { final String subscriptionName = Codec.decode(cursor.getName()); - subscriptions.put(subscriptionName, createPersistentSubscription(subscriptionName, cursor, - PersistentSubscription.isCursorFromReplicatedSubscription(cursor) ? true : null, - cursor.getCursorProperties())); + Optional replicatedSubscriptionConfiguration = + PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor); + Boolean replicated = replicatedSubscriptionConfiguration.orElse(null); + subscriptions.put(subscriptionName, + createPersistentSubscription(subscriptionName, cursor, replicated, + cursor.getCursorProperties())); // subscription-cursor gets activated by default: deactivate as there is no active subscription // right now subscriptions.get(subscriptionName).deactivateCursor(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java index 9a46a2919d12a..360be2e435ab1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doCallRealMethod; @@ -30,11 +31,13 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.Position; @@ -226,6 +229,40 @@ public void testAcknowledgeUpdateCursorLastActive() throws Exception { assertTrue(persistentSubscription.cursor.getLastActive() > beforeAcknowledgeTimestamp); } + @Test + public void testGetReplicatedSubscriptionConfiguration() { + Map properties = PersistentSubscription.getBaseCursorProperties(true); + assertThat(properties).containsEntry(PersistentSubscription.REPLICATED_SUBSCRIPTION_PROPERTY, 1L); + ManagedCursor cursor = mock(ManagedCursor.class); + doReturn(properties).when(cursor).getProperties(); + Optional replicatedSubscriptionConfiguration = + PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor); + assertThat(replicatedSubscriptionConfiguration).isNotEmpty().get().isEqualTo(Boolean.TRUE); + + properties = Map.of(PersistentSubscription.REPLICATED_SUBSCRIPTION_PROPERTY, 10L); + doReturn(properties).when(cursor).getProperties(); + replicatedSubscriptionConfiguration = + PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor); + assertThat(replicatedSubscriptionConfiguration).isEmpty(); + properties = Map.of(PersistentSubscription.REPLICATED_SUBSCRIPTION_PROPERTY, -1L); + doReturn(properties).when(cursor).getProperties(); + replicatedSubscriptionConfiguration = + PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor); + assertThat(replicatedSubscriptionConfiguration).isEmpty(); + + properties = PersistentSubscription.getBaseCursorProperties(false); + doReturn(properties).when(cursor).getProperties(); + replicatedSubscriptionConfiguration = + PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor); + assertThat(replicatedSubscriptionConfiguration).isEmpty(); + + properties = PersistentSubscription.getBaseCursorProperties(null); + doReturn(properties).when(cursor).getProperties(); + replicatedSubscriptionConfiguration = + PersistentSubscription.getReplicatedSubscriptionConfiguration(cursor); + assertThat(replicatedSubscriptionConfiguration).isEmpty(); + } + public static class CustomTransactionPendingAckStoreProvider implements TransactionPendingAckStoreProvider { @Override public CompletableFuture newPendingAckStore(PersistentSubscription subscription) { From 985060576bc9e290793c5815ff5f25f305c1515e Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 31 Dec 2024 15:32:13 +0800 Subject: [PATCH 227/327] [fix][broker] Skip to persist cursor info if it failed by cursor closed (#23615) --- .../apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 0cd9fc0d54cc8..934bfba4b0d81 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -3254,6 +3254,13 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin mbean.addWriteCursorLedgerSize(data.length); callback.operationComplete(); } else { + if (state == State.Closed) { + // After closed the cursor, the in-progress persistence task will get a + // BKException.Code.LedgerClosedException. + callback.operationFailed(new CursorAlreadyClosedException(String.format("%s %s skipped this" + + " persistence, because the cursor already closed", ledger.getName(), name))); + return; + } log.warn("[{}] Error updating cursor {} position {} in meta-ledger {}: {}", ledger.getName(), name, position, lh1.getId(), BKException.getMessage(rc)); // If we've had a write error, the ledger will be automatically closed, we need to create a new one, From 7619e2fb4d44f05b64ec7d050a5202f974c621c2 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 2 Jan 2025 15:14:12 +0800 Subject: [PATCH 228/327] [fix][broker] Continue using the next provider for authentication if one fails (#23797) Signed-off-by: Zixuan Liu --- .../AuthenticationProviderList.java | 60 ++++---- .../AuthenticationProviderListTest.java | 129 +++++++++++++++++- 2 files changed, 154 insertions(+), 35 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java index 0e5559b3c3aab..02ed52b5ec042 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java @@ -42,7 +42,7 @@ public class AuthenticationProviderList implements AuthenticationProvider { private interface AuthProcessor { - T apply(W process) throws AuthenticationException; + T apply(W process) throws Exception; } @@ -51,21 +51,30 @@ private enum ErrorCode { AUTH_REQUIRED, } + private static AuthenticationException newAuthenticationException(String message, Exception e) { + AuthenticationException authenticationException = new AuthenticationException(message); + if (e != null) { + authenticationException.initCause(e); + } + return authenticationException; + } + private static T applyAuthProcessor(List processors, AuthenticationMetrics metrics, AuthProcessor authFunc) throws AuthenticationException { - AuthenticationException authenticationException = null; + Exception authenticationException = null; String errorCode = ErrorCode.UNKNOWN.name(); for (W ap : processors) { try { return authFunc.apply(ap); - } catch (AuthenticationException ae) { + } catch (Exception ae) { if (log.isDebugEnabled()) { log.debug("Authentication failed for auth provider " + ap.getClass() + ": ", ae); } - // Store the exception so we can throw it later instead of a generic one authenticationException = ae; - errorCode = ap.getClass().getSimpleName() + "-INVALID-AUTH"; + if (ae instanceof AuthenticationException) { + errorCode = ap.getClass().getSimpleName() + "-INVALID-AUTH"; + } } } @@ -76,7 +85,7 @@ private static T applyAuthProcessor(List processors, AuthenticationMet } else { metrics.recordFailure(AuthenticationProviderList.class.getSimpleName(), "authentication-provider-list", errorCode); - throw authenticationException; + throw newAuthenticationException("Authentication failed", authenticationException); } } @@ -290,12 +299,12 @@ public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteA throws AuthenticationException { final List states = new ArrayList<>(providers.size()); - AuthenticationException authenticationException = null; + Exception authenticationException = null; for (AuthenticationProvider provider : providers) { try { AuthenticationState state = provider.newAuthState(authData, remoteAddress, sslSession); states.add(state); - } catch (AuthenticationException ae) { + } catch (Exception ae) { if (log.isDebugEnabled()) { log.debug("Authentication failed for auth provider " + provider.getClass() + ": ", ae); } @@ -305,11 +314,8 @@ public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteA } if (states.isEmpty()) { log.debug("Failed to initialize a new auth state from {}", remoteAddress, authenticationException); - if (authenticationException != null) { - throw authenticationException; - } else { - throw new AuthenticationException("Failed to initialize a new auth state from " + remoteAddress); - } + throw newAuthenticationException("Failed to initialize a new auth state from " + remoteAddress, + authenticationException); } else { return new AuthenticationListState(states, authenticationMetrics); } @@ -319,12 +325,12 @@ public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteA public AuthenticationState newHttpAuthState(HttpServletRequest request) throws AuthenticationException { final List states = new ArrayList<>(providers.size()); - AuthenticationException authenticationException = null; + Exception authenticationException = null; for (AuthenticationProvider provider : providers) { try { AuthenticationState state = provider.newHttpAuthState(request); states.add(state); - } catch (AuthenticationException ae) { + } catch (Exception ae) { if (log.isDebugEnabled()) { log.debug("Authentication failed for auth provider " + provider.getClass() + ": ", ae); } @@ -335,12 +341,9 @@ public AuthenticationState newHttpAuthState(HttpServletRequest request) throws A if (states.isEmpty()) { log.debug("Failed to initialize a new http auth state from {}", request.getRemoteHost(), authenticationException); - if (authenticationException != null) { - throw authenticationException; - } else { - throw new AuthenticationException( - "Failed to initialize a new http auth state from " + request.getRemoteHost()); - } + throw newAuthenticationException( + "Failed to initialize a new http auth state from " + request.getRemoteHost(), + authenticationException); } else { return new AuthenticationListState(states, authenticationMetrics); } @@ -348,22 +351,11 @@ public AuthenticationState newHttpAuthState(HttpServletRequest request) throws A @Override public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletResponse response) throws Exception { - Boolean authenticated = applyAuthProcessor( + return applyAuthProcessor( providers, authenticationMetrics, - provider -> { - try { - return provider.authenticateHttpRequest(request, response); - } catch (Exception e) { - if (e instanceof AuthenticationException) { - throw (AuthenticationException) e; - } else { - throw new AuthenticationException("Failed to authentication http request"); - } - } - } + provider -> provider.authenticateHttpRequest(request, response) ); - return authenticated; } @Override diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java index e81198217b5b6..f139bb384a4be 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java @@ -21,6 +21,8 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedDataAttributeName; import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedRoleAttributeName; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.isA; import static org.mockito.Mockito.mock; @@ -35,13 +37,17 @@ import java.security.KeyPair; import java.security.PrivateKey; import java.util.Date; +import java.util.List; import java.util.Optional; import java.util.Properties; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import javax.naming.AuthenticationException; import javax.servlet.http.HttpServletRequest; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.common.api.AuthData; +import org.apache.pulsar.common.util.FutureUtil; import org.assertj.core.util.Lists; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -260,4 +266,125 @@ public void testAuthenticateHttpRequest() throws Exception { verify(requestBB).setAttribute(eq(AuthenticatedDataAttributeName), isA(AuthenticationDataSource.class)); } -} + @Test + public void testAuthenticateWithMultipleProviders() throws Exception { + HttpServletRequest httpRequest = mock(HttpServletRequest.class); + AuthenticationDataSource authenticationDataSource = mock(AuthenticationDataSource.class); + + AuthenticationProvider failingProvider = mock(AuthenticationProvider.class); + List providers = Lists.newArrayList( + failingProvider + ); + try (AuthenticationProvider provider = new AuthenticationProviderList(providers)) { + provider.initialize(new ServiceConfiguration()); + RuntimeException authenticateException = new RuntimeException("authenticateException"); + + when(failingProvider.authenticateAsync(authenticationDataSource)) + .thenReturn(FutureUtil.failedFuture(authenticateException)); + when(failingProvider.authenticate(authenticationDataSource)) + .thenThrow(authenticateException); + assertThat(provider.authenticateAsync(authenticationDataSource)) + .failsWithin(3, TimeUnit.SECONDS) + .withThrowableThat().withCause(authenticateException); + assertThatThrownBy(() -> provider.authenticate(authenticationDataSource)) + .isInstanceOf(AuthenticationException.class) + .hasCause(authenticateException); + + RuntimeException authenticateHttpRequestException = new RuntimeException("authenticateHttpRequestAsync"); + when(failingProvider.authenticateHttpRequestAsync(httpRequest, null)) + .thenReturn(FutureUtil.failedFuture(authenticateHttpRequestException)); + when(failingProvider.authenticateHttpRequest(httpRequest, null)) + .thenThrow(authenticateHttpRequestException); + assertThat(provider.authenticateHttpRequestAsync(httpRequest, null)) + .failsWithin(3, TimeUnit.SECONDS) + .withThrowableThat() + .havingCause() + .withCause(authenticateHttpRequestException); + assertThatThrownBy(() -> provider.authenticateHttpRequest(httpRequest, null)) + .isInstanceOf(AuthenticationException.class) + .hasCause(authenticateHttpRequestException); + + RuntimeException newAuthStateException = new RuntimeException("newAuthState"); + when(failingProvider.newAuthState(null, null, null)) + .thenThrow(newAuthStateException); + assertThatThrownBy(() -> provider.newAuthState(null, null, null)) + .isInstanceOf(AuthenticationException.class) + .hasCause(newAuthStateException); + + RuntimeException newHttpAuthStateException = new RuntimeException("newHttpAuthState"); + when(failingProvider.newHttpAuthState(httpRequest)) + .thenThrow(newHttpAuthStateException); + assertThatThrownBy(() -> provider.newHttpAuthState(httpRequest)) + .isInstanceOf(AuthenticationException.class) + .hasCause(newHttpAuthStateException); + } + + AuthenticationProvider successfulProvider = mock(AuthenticationProvider.class); + providers.add(successfulProvider); + String subject = "test-role"; + + try (AuthenticationProvider provider = new AuthenticationProviderList(providers)) { + provider.initialize(new ServiceConfiguration()); + + when(successfulProvider.authenticateAsync(authenticationDataSource)) + .thenReturn(CompletableFuture.completedFuture(subject)); + when(successfulProvider.authenticate(authenticationDataSource)) + .thenReturn(subject); + assertThat(provider.authenticateAsync(authenticationDataSource)) + .succeedsWithin(3, TimeUnit.SECONDS) + .matches(subject::equals); + assertThat(provider.authenticate(authenticationDataSource)) + .isEqualTo(subject); + + when(successfulProvider.authenticateHttpRequestAsync(httpRequest, null)) + .thenReturn(CompletableFuture.completedFuture(true)); + when(successfulProvider.authenticateHttpRequest(httpRequest, null)) + .thenReturn(true); + assertThat(provider.authenticateHttpRequestAsync(httpRequest, null)) + .succeedsWithin(3, TimeUnit.SECONDS) + .isEqualTo(true); + assertThat(provider.authenticateHttpRequest(httpRequest, null)) + .isEqualTo(true); + + AuthenticationState authenticationState = new AuthenticationState() { + @Override + public String getAuthRole() { + return subject; + } + + @Override + public AuthData authenticate(AuthData authData) { + return null; + } + + @Override + public AuthenticationDataSource getAuthDataSource() { + return null; + } + + @Override + public boolean isComplete() { + return false; + } + }; + when(successfulProvider.newAuthState(null, null, null)) + .thenReturn(authenticationState); + when(successfulProvider.newHttpAuthState(httpRequest)).thenReturn(authenticationState); + verifyAuthenticationStateSuccess(provider.newAuthState(null, null, null), true, subject); + verifyAuthenticationStateSuccess(provider.newAuthState(null, null, null), false, subject); + verifyAuthenticationStateSuccess(provider.newHttpAuthState(httpRequest), true, subject); + verifyAuthenticationStateSuccess(provider.newHttpAuthState(httpRequest), false, subject); + } + } + + private void verifyAuthenticationStateSuccess(AuthenticationState authState, boolean isAsync, String expectedRole) + throws Exception { + assertThat(authState).isNotNull(); + if (isAsync) { + assertThat(authState.authenticateAsync(null)).succeedsWithin(3, TimeUnit.SECONDS); + } else { + assertThat(authState.authenticate(null)).isNull(); + } + assertThat(authState.getAuthRole()).isEqualTo(expectedRole); + } +} \ No newline at end of file From b02d52ca8c8afd62681c0e243d16d8958abb5380 Mon Sep 17 00:00:00 2001 From: zjxxzjwang Date: Thu, 2 Jan 2025 15:16:24 +0800 Subject: [PATCH 229/327] [feat][broker] Prevent auto-creation of topics using legacy cluster-based naming scheme (#23620) Co-authored-by: zjxxzjwang --- conf/broker.conf | 4 ++++ conf/standalone.conf | 4 ++++ .../pulsar/broker/ServiceConfiguration.java | 5 +++++ .../pulsar/broker/service/BrokerService.java | 7 +++++++ .../BrokerServiceAutoTopicCreationTest.java | 19 +++++++++++++++++++ 5 files changed, 39 insertions(+) diff --git a/conf/broker.conf b/conf/broker.conf index af335c141534f..f68306ec7b4d7 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -198,6 +198,10 @@ allowAutoTopicCreation=true # The type of topic that is allowed to be automatically created.(partitioned/non-partitioned) allowAutoTopicCreationType=non-partitioned +# If 'allowAutoTopicCreation' is true and the name of the topic contains 'cluster', +# the topic cannot be automatically created. +allowAutoTopicCreationWithLegacyNamingScheme=true + # Enable subscription auto creation if new consumer connected (disable auto creation with value false) allowAutoSubscriptionCreation=true diff --git a/conf/standalone.conf b/conf/standalone.conf index 90cf3b57ff941..2036556da4385 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -1181,6 +1181,10 @@ allowAutoTopicCreation=true # The type of topic that is allowed to be automatically created.(partitioned/non-partitioned) allowAutoTopicCreationType=non-partitioned +# If 'allowAutoTopicCreation' is true and the name of the topic contains 'cluster', +# the topic cannot be automatically created. +allowAutoTopicCreationWithLegacyNamingScheme=true + # Enable subscription auto creation if new consumer connected (disable auto creation with value false) allowAutoSubscriptionCreation=true diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 0f7ae00713dce..0b6f0e9418cf9 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2154,6 +2154,11 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "The type of topic that is allowed to be automatically created.(partitioned/non-partitioned)" ) private TopicType allowAutoTopicCreationType = TopicType.NON_PARTITIONED; + @FieldContext(category = CATEGORY_SERVER, dynamic = true, + doc = "If 'allowAutoTopicCreation' is true and the name of the topic contains 'cluster'," + + "the topic cannot be automatically created." + ) + private boolean allowAutoTopicCreationWithLegacyNamingScheme = true; @FieldContext( category = CATEGORY_STORAGE_ML, dynamic = true, 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 c79d839097e68..b8102488f493d 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 @@ -3536,6 +3536,13 @@ private CompletableFuture isAllowAutoTopicCreationAsync(final TopicName return CompletableFuture.completedFuture(true); } + //If 'allowAutoTopicCreation' is true, and the name of the topic contains 'cluster', + //the topic cannot be automatically created. + if (!pulsar.getConfiguration().isAllowAutoTopicCreationWithLegacyNamingScheme() + && StringUtils.isNotBlank(topicName.getCluster())) { + return CompletableFuture.completedFuture(false); + } + final boolean allowed; AutoTopicCreationOverride autoTopicCreationOverride = getAutoTopicCreationOverride(topicName, policies); if (autoTopicCreationOverride != null) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java index 3e735ee4c85b8..59eb8ab74c863 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java @@ -587,4 +587,23 @@ public void testExtensibleLoadManagerImplInternalTopicAutoCreations() } + @Test + public void testAutoPartitionedTopicNameWithClusterName() throws Exception { + pulsar.getConfiguration().setAllowAutoTopicCreation(true); + pulsar.getConfiguration().setAllowAutoTopicCreationType(TopicType.PARTITIONED); + pulsar.getConfiguration().setDefaultNumPartitions(3); + + final String topicString = "persistent://prop/ns-abc/testTopic/1"; + // When allowAutoTopicCreationWithLegacyNamingScheme as the default value is false, + // four-paragraph topic cannot be created. + pulsar.getConfiguration().setAllowAutoTopicCreationWithLegacyNamingScheme(false); + Assert.assertThrows(PulsarClientException.NotFoundException.class, + () -> pulsarClient.newProducer().topic(topicString).create()); + + pulsar.getConfiguration().setAllowAutoTopicCreationWithLegacyNamingScheme(true); + Producer producer = pulsarClient.newProducer().topic(topicString).create(); + Assert.assertEquals(producer.getTopic(), topicString); + producer.close(); + } + } From 4a01423273c010eebce549139b2fc400bb59555e Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 2 Jan 2025 15:54:34 +0800 Subject: [PATCH 230/327] [fix][broker] Msg delivery is stuck due to items in the collection recentlyJoinedConsumers are out-of-order (#23795) --- ...KeyDispatcherMultipleConsumersClassic.java | 40 ++++++++- ...ispatcherMultipleConsumersClassicTest.java | 81 +++++++++++++++++++ 2 files changed, 119 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java index c227bf5b435bc..71f37c5939d6a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java @@ -146,11 +146,44 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { && consumerList.size() > 1 && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) { recentlyJoinedConsumers.put(consumer, readPositionWhenJoining); + sortRecentlyJoinedConsumersIfNeeded(); } } }); } + private void sortRecentlyJoinedConsumersIfNeeded() { + if (recentlyJoinedConsumers.size() == 1) { + return; + } + boolean sortNeeded = false; + Position posPre = null; + Position posAfter = null; + for (Map.Entry entry : recentlyJoinedConsumers.entrySet()) { + if (posPre == null) { + posPre = entry.getValue(); + } else { + posAfter = entry.getValue(); + } + if (posPre != null && posAfter != null) { + if (posPre.compareTo(posAfter) > 0) { + sortNeeded = true; + break; + } + posPre = posAfter; + } + } + + if (sortNeeded) { + List> sortedList = new ArrayList<>(recentlyJoinedConsumers.entrySet()); + Collections.sort(sortedList, Map.Entry.comparingByValue()); + recentlyJoinedConsumers.clear(); + for (Map.Entry entry : sortedList) { + recentlyJoinedConsumers.put(entry.getKey(), entry.getValue()); + } + } + } + @Override public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { // The consumer must be removed from the selector before calling the superclass removeConsumer method. @@ -560,8 +593,11 @@ public boolean hasSameKeySharedPolicy(KeySharedMeta ksm) { && ksm.isAllowOutOfOrderDelivery() == this.allowOutOfOrderDelivery); } - public LinkedHashMap getRecentlyJoinedConsumers() { - return recentlyJoinedConsumers; + public synchronized LinkedHashMap getRecentlyJoinedConsumers() { + if (recentlyJoinedConsumers == null) { + return null; + } + return new LinkedHashMap<>(recentlyJoinedConsumers); } public Map> getConsumerKeyHashRanges() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java index 1f40fd46aa344..af42fc3dca402 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java @@ -45,7 +45,9 @@ import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Collections; +import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Queue; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; @@ -182,6 +184,85 @@ public void testAddConsumerWhenClosed() throws Exception { assertTrue(persistentDispatcher.getSelector().getConsumerKeyHashRanges().isEmpty()); } + @Test + public void testSortRecentlyJoinedConsumersIfNeeded() throws Exception { + PersistentStickyKeyDispatcherMultipleConsumersClassic persistentDispatcher = + new PersistentStickyKeyDispatcherMultipleConsumersClassic( + topicMock, cursorMock, subscriptionMock, configMock, + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)); + + Consumer consumer0 = mock(Consumer.class); + when(consumer0.consumerName()).thenReturn("c0-1"); + Consumer consumer1 = mock(Consumer.class); + when(consumer1.consumerName()).thenReturn("c1"); + Consumer consumer2 = mock(Consumer.class); + when(consumer2.consumerName()).thenReturn("c2"); + Consumer consumer3 = mock(Consumer.class); + when(consumer3.consumerName()).thenReturn("c3"); + Consumer consumer4 = mock(Consumer.class); + when(consumer4.consumerName()).thenReturn("c4"); + Consumer consumer5 = mock(Consumer.class); + when(consumer5.consumerName()).thenReturn("c5"); + Consumer consumer6 = mock(Consumer.class); + when(consumer6.consumerName()).thenReturn("c6"); + + when(cursorMock.getNumberOfEntriesSinceFirstNotAckedMessage()).thenReturn(100L); + when(cursorMock.getMarkDeletedPosition()).thenReturn(PositionFactory.create(-1, -1)); + + when(cursorMock.getReadPosition()).thenReturn(PositionFactory.create(0, 0)); + persistentDispatcher.addConsumer(consumer0).join(); + + when(cursorMock.getReadPosition()).thenReturn(PositionFactory.create(4, 1)); + persistentDispatcher.addConsumer(consumer1).join(); + + when(cursorMock.getReadPosition()).thenReturn(PositionFactory.create(5, 2)); + persistentDispatcher.addConsumer(consumer2).join(); + + when(cursorMock.getReadPosition()).thenReturn(PositionFactory.create(5, 1)); + persistentDispatcher.addConsumer(consumer3).join(); + + when(cursorMock.getReadPosition()).thenReturn(PositionFactory.create(5, 3)); + persistentDispatcher.addConsumer(consumer4).join(); + + when(cursorMock.getReadPosition()).thenReturn(PositionFactory.create(4, 2)); + persistentDispatcher.addConsumer(consumer5).join(); + + when(cursorMock.getReadPosition()).thenReturn(PositionFactory.create(6, 1)); + persistentDispatcher.addConsumer(consumer6).join(); + + assertEquals(persistentDispatcher.getRecentlyJoinedConsumers().size(), 6); + + Iterator> itr + = persistentDispatcher.getRecentlyJoinedConsumers().entrySet().iterator(); + + Map.Entry entry1 = itr.next(); + assertEquals(entry1.getValue(), PositionFactory.create(4, 1)); + assertEquals(entry1.getKey(), consumer1); + + Map.Entry entry2 = itr.next(); + assertEquals(entry2.getValue(), PositionFactory.create(4, 2)); + assertEquals(entry2.getKey(), consumer5); + + Map.Entry entry3 = itr.next(); + assertEquals(entry3.getValue(), PositionFactory.create(5, 1)); + assertEquals(entry3.getKey(), consumer3); + + Map.Entry entry4 = itr.next(); + assertEquals(entry4.getValue(), PositionFactory.create(5, 2)); + assertEquals(entry4.getKey(), consumer2); + + Map.Entry entry5 = itr.next(); + assertEquals(entry5.getValue(), PositionFactory.create(5, 3)); + assertEquals(entry5.getKey(), consumer4); + + Map.Entry entry6 = itr.next(); + assertEquals(entry6.getValue(), PositionFactory.create(6, 1)); + assertEquals(entry6.getKey(), consumer6); + + // cleanup. + persistentDispatcher.close(); + } + @Test public void testSendMarkerMessage() { try { From fd4502905089272882e7c0bda494170dafd60bda Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 2 Jan 2025 16:16:08 +0800 Subject: [PATCH 231/327] [fix][client] Fix compatibility between kerberos and tls (#23798) Signed-off-by: Zixuan Liu --- .../client/api/TlsProducerConsumerTest.java | 69 ++++++++++++++++++ .../internal/http/AsyncHttpConnector.java | 7 +- .../pulsar/client/api/Authentication.java | 1 + .../apache/pulsar/client/cli/CmdConsume.java | 2 +- .../apache/pulsar/client/cli/CmdProduce.java | 2 +- .../org/apache/pulsar/client/cli/CmdRead.java | 2 +- .../apache/pulsar/client/impl/HttpClient.java | 7 +- .../client/impl/PulsarChannelInitializer.java | 70 +++++++++++-------- .../client/impl/ClientInitializationTest.java | 2 +- .../proxy/server/AdminProxyHandler.java | 32 +++++---- .../proxy/server/DirectProxyHandler.java | 64 +++++++++-------- .../server/ProxyServiceTlsStarterTest.java | 2 + .../socket/client/PerformanceClient.java | 2 +- 13 files changed, 182 insertions(+), 80 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java index 44af37ca90f51..98b917330ffd8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java @@ -18,18 +18,24 @@ */ package org.apache.pulsar.client.api; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.util.Arrays; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; import lombok.Cleanup; import org.apache.commons.compress.utils.IOUtils; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -295,4 +301,67 @@ public void testTlsTransport(Supplier url, Authentication auth) throws E @Cleanup Producer ignored = client.newProducer().topic(topicName).create(); } + + @Test + public void testTlsWithFakeAuthentication() throws Exception { + Authentication authentication = spy(new Authentication() { + @Override + public String getAuthMethodName() { + return "fake"; + } + + @Override + public void configure(Map authParams) { + + } + + @Override + public void start() { + + } + + @Override + public void close() { + + } + + @Override + public AuthenticationDataProvider getAuthData(String brokerHostName) { + return mock(AuthenticationDataProvider.class); + } + }); + + @Cleanup + PulsarAdmin pulsarAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsar().getWebServiceAddressTls()) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .allowTlsInsecureConnection(false) + .enableTlsHostnameVerification(false) + .tlsKeyFilePath(getTlsFileForClient("admin.key-pk8")) + .tlsCertificateFilePath(getTlsFileForClient("admin.cert")) + .authentication(authentication) + .build(); + pulsarAdmin.tenants().getTenants(); + verify(authentication, never()).getAuthData(); + + @Cleanup + PulsarClient pulsarClient = PulsarClient.builder().serviceUrl(getPulsar().getBrokerServiceUrlTls()) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .allowTlsInsecureConnection(false) + .enableTlsHostnameVerification(false) + .tlsKeyFilePath(getTlsFileForClient("admin.key-pk8")) + .tlsCertificateFilePath(getTlsFileForClient("admin.cert")) + .authentication(authentication).build(); + verify(authentication, never()).getAuthData(); + + final String topicName = "persistent://my-property/my-ns/my-topic-1"; + internalSetUpForNamespace(); + @Cleanup + Consumer ignoredConsumer = + pulsarClient.newConsumer().topic(topicName).subscriptionName("my-subscriber-name").subscribe(); + verify(authentication, never()).getAuthData(); + @Cleanup + Producer ignoredProducer = pulsarClient.newProducer().topic(topicName).create(); + verify(authentication, never()).getAuthData(); + } } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java index 64ba0e99cb915..fb11d9e46d34c 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java @@ -191,7 +191,8 @@ private void configureAsyncHttpClientSslEngineFactory(ClientConfigurationData co // Set client key and certificate if available sslRefresher = Executors.newScheduledThreadPool(1, new DefaultThreadFactory("pulsar-admin-ssl-refresher")); - PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf, serviceNameResolver + .resolveHostUri().getHost()); this.sslFactory = (PulsarSslFactory) Class.forName(conf.getSslFactoryPlugin()) .getConstructor().newInstance(); this.sslFactory.initialize(sslConfiguration); @@ -519,7 +520,7 @@ public void close() { } } - protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData conf) + protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData conf, String host) throws PulsarClientException { return PulsarSslConfiguration.builder() .tlsProvider(conf.getSslProvider()) @@ -537,7 +538,7 @@ protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData c .allowInsecureConnection(conf.isTlsAllowInsecureConnection()) .requireTrustedClientCertOnConnect(false) .tlsEnabledWithKeystore(conf.isUseKeyStoreTls()) - .authData(conf.getAuthentication().getAuthData()) + .authData(conf.getAuthentication().getAuthData(host)) .tlsCustomParams(conf.getSslFactoryPluginParams()) .serverMode(false) .isHttps(true) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Authentication.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Authentication.java index 9bf1b24cbdb32..48d9e3e230701 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Authentication.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Authentication.java @@ -48,6 +48,7 @@ public interface Authentication extends Closeable, Serializable { * @throws PulsarClientException * any other error */ + @Deprecated default AuthenticationDataProvider getAuthData() throws PulsarClientException { throw new UnsupportedAuthenticationException("Method not implemented!"); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java index 0f0e2f0a9c813..98ca9bc814960 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java @@ -260,7 +260,7 @@ private int consumeFromWebSocket(String topic) { try { if (authentication != null) { authentication.start(); - AuthenticationDataProvider authData = authentication.getAuthData(); + AuthenticationDataProvider authData = authentication.getAuthData(consumerUri.getHost()); if (authData.hasDataForHttp()) { for (Map.Entry kv : authData.getHttpHeaders()) { consumeRequest.setHeader(kv.getKey(), kv.getValue()); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java index e5a8836602151..01ac1df333dcf 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java @@ -469,7 +469,7 @@ private int publishToWebSocket(String topic) { try { if (authentication != null) { authentication.start(); - AuthenticationDataProvider authData = authentication.getAuthData(); + AuthenticationDataProvider authData = authentication.getAuthData(produceUri.getHost()); if (authData.hasDataForHttp()) { for (Map.Entry kv : authData.getHttpHeaders()) { produceRequest.setHeader(kv.getKey(), kv.getValue()); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java index 529d1d9c41272..d0e20dfa57083 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java @@ -244,7 +244,7 @@ private int readFromWebSocket(String topic) { try { if (authentication != null) { authentication.start(); - AuthenticationDataProvider authData = authentication.getAuthData(); + AuthenticationDataProvider authData = authentication.getAuthData(readerUri.getHost()); if (authData.hasDataForHttp()) { for (Map.Entry kv : authData.getHttpHeaders()) { readRequest.setHeader(kv.getKey(), kv.getValue()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java index c53f79af12e75..8e448d801fa6b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java @@ -97,7 +97,8 @@ public boolean keepAlive(InetSocketAddress remoteAddress, Request ahcRequest, this.executorService = Executors .newSingleThreadScheduledExecutor(new ExecutorProvider .ExtendedThreadFactory("httpclient-ssl-refresh")); - PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf); + PulsarSslConfiguration sslConfiguration = + buildSslConfiguration(conf, serviceNameResolver.resolveHostUri().getHost()); this.sslFactory = (PulsarSslFactory) Class.forName(conf.getSslFactoryPlugin()) .getConstructor().newInstance(); this.sslFactory.initialize(sslConfiguration); @@ -233,7 +234,7 @@ public CompletableFuture get(String path, Class clazz) { return future; } - protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData config) + protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData config, String host) throws PulsarClientException { return PulsarSslConfiguration.builder() .tlsProvider(config.getSslProvider()) @@ -252,7 +253,7 @@ protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData c .requireTrustedClientCertOnConnect(false) .tlsEnabledWithKeystore(config.isUseKeyStoreTls()) .tlsCustomParams(config.getSslFactoryPluginParams()) - .authData(config.getAuthentication().getAuthData()) + .authData(config.getAuthentication().getAuthData(host)) .serverMode(false) .isHttps(true) .build(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java index 5097c34e0b2fd..b20833e46a278 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java @@ -27,8 +27,10 @@ import io.netty.handler.proxy.Socks5ProxyHandler; import io.netty.handler.ssl.SslHandler; import java.net.InetSocketAddress; +import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; @@ -55,8 +57,8 @@ public class PulsarChannelInitializer extends ChannelInitializer private final InetSocketAddress socks5ProxyAddress; private final String socks5ProxyUsername; private final String socks5ProxyPassword; - - private final PulsarSslFactory pulsarSslFactory; + private final ClientConfigurationData conf; + private final Map pulsarSslFactoryMap; private static final long TLS_CERTIFICATE_CACHE_MILLIS = TimeUnit.MINUTES.toMillis(1); @@ -69,26 +71,17 @@ public PulsarChannelInitializer(ClientConfigurationData conf, Supplier(); if (scheduledExecutorService != null && conf.getAutoCertRefreshSeconds() > 0) { scheduledExecutorService.scheduleWithFixedDelay(() -> this.refreshSslContext(conf), conf.getAutoCertRefreshSeconds(), conf.getAutoCertRefreshSeconds(), TimeUnit.SECONDS); } - } else { - pulsarSslFactory = null; + this.pulsarSslFactoryMap = null; } } @@ -123,6 +116,23 @@ CompletableFuture initTls(Channel ch, InetSocketAddress sniHost) { CompletableFuture initTlsFuture = new CompletableFuture<>(); ch.eventLoop().execute(() -> { try { + PulsarSslFactory pulsarSslFactory = pulsarSslFactoryMap.computeIfAbsent(sniHost.getHostName(), key -> { + try { + PulsarSslFactory factory = (PulsarSslFactory) Class.forName(conf.getSslFactoryPlugin()) + .getConstructor().newInstance(); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf, key); + factory.initialize(sslConfiguration); + factory.createInternalSslContext(); + return factory; + } catch (Exception e) { + log.error("Unable to initialize and create the ssl context", e); + initTlsFuture.completeExceptionally(e); + return null; + } + }); + if (pulsarSslFactory == null) { + return; + } SslHandler handler = new SslHandler(pulsarSslFactory .createClientSslEngine(ch.alloc(), sniHost.getHostName(), sniHost.getPort())); @@ -181,7 +191,9 @@ CompletableFuture initializeClientCnx(Channel ch, return ch; })); } - protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData config) + +protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData config, + String host) throws PulsarClientException { return PulsarSslConfiguration.builder() .tlsProvider(config.getSslProvider()) @@ -200,28 +212,30 @@ protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData c .requireTrustedClientCertOnConnect(false) .tlsEnabledWithKeystore(config.isUseKeyStoreTls()) .tlsCustomParams(config.getSslFactoryPluginParams()) - .authData(config.getAuthentication().getAuthData()) + .authData(config.getAuthentication().getAuthData(host)) .serverMode(false) .build(); } protected void refreshSslContext(ClientConfigurationData conf) { - try { + pulsarSslFactoryMap.forEach((key, pulsarSslFactory) -> { try { - if (conf.isUseKeyStoreTls()) { - this.pulsarSslFactory.getInternalSslContext(); - } else { - this.pulsarSslFactory.getInternalNettySslContext(); + try { + if (conf.isUseKeyStoreTls()) { + pulsarSslFactory.getInternalSslContext(); + } else { + pulsarSslFactory.getInternalNettySslContext(); + } + } catch (Exception e) { + log.error("SSL Context is not initialized", e); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf, key); + pulsarSslFactory.initialize(sslConfiguration); } + pulsarSslFactory.update(); } catch (Exception e) { - log.error("SSL Context is not initialized", e); - PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf); - this.pulsarSslFactory.initialize(sslConfiguration); + log.error("Failed to refresh SSL context", e); } - this.pulsarSslFactory.update(); - } catch (Exception e) { - log.error("Failed to refresh SSL context", e); - } + }); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientInitializationTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientInitializationTest.java index 2682d011cd0c5..f7ff30c286c76 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientInitializationTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientInitializationTest.java @@ -41,6 +41,6 @@ public void testInitializeAuthWithTls() throws PulsarClientException { .build(); verify(auth).start(); - verify(auth, times(1)).getAuthData(); + verify(auth, times(0)).getAuthData(); } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java index 54b6db5198c57..8d0c2c3ae397a 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java @@ -38,6 +38,7 @@ import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.web.AuthenticationFilter; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; @@ -290,6 +291,19 @@ protected HttpClient newHttpClient() { return new JettyHttpClient(); } + private String getWebServiceUrl() throws PulsarServerException { + if (isBlank(brokerWebServiceUrl)) { + ServiceLookupData availableBroker = discoveryProvider.nextBroker(); + if (config.isTlsEnabledWithBroker()) { + return availableBroker.getWebServiceUrlTls(); + } else { + return availableBroker.getWebServiceUrl(); + } + } else { + return brokerWebServiceUrl; + } + } + @Override protected String rewriteTarget(HttpServletRequest request) { StringBuilder url = new StringBuilder(); @@ -305,17 +319,10 @@ protected String rewriteTarget(HttpServletRequest request) { if (isFunctionsRestRequest && !isBlank(functionWorkerWebServiceUrl)) { url.append(functionWorkerWebServiceUrl); - } else if (isBlank(brokerWebServiceUrl)) { + } else { try { - ServiceLookupData availableBroker = discoveryProvider.nextBroker(); - - if (config.isTlsEnabledWithBroker()) { - url.append(availableBroker.getWebServiceUrlTls()); - } else { - url.append(availableBroker.getWebServiceUrl()); - } - - if (LOG.isDebugEnabled()) { + url.append(getWebServiceUrl()); + if (LOG.isDebugEnabled() && isBlank(brokerWebServiceUrl)) { LOG.debug("[{}:{}] Selected active broker is {}", request.getRemoteAddr(), request.getRemotePort(), url); } @@ -324,8 +331,6 @@ protected String rewriteTarget(HttpServletRequest request) { request.getRemotePort(), e.getMessage(), e); return null; } - } else { - url.append(brokerWebServiceUrl); } if (url.lastIndexOf("/") == url.length() - 1) { @@ -398,7 +403,8 @@ protected PulsarSslConfiguration buildSslConfiguration(AuthenticationDataProvide protected PulsarSslFactory createPulsarSslFactory() { try { try { - AuthenticationDataProvider authData = proxyClientAuthentication.getAuthData(); + AuthenticationDataProvider authData = + proxyClientAuthentication.getAuthData(URI.create(getWebServiceUrl()).getHost()); PulsarSslConfiguration pulsarSslConfiguration = buildSslConfiguration(authData); PulsarSslFactory sslFactory = (PulsarSslFactory) Class.forName(config.getBrokerClientSslFactoryPlugin()) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java index 407c93074a0fc..681aa553c48a8 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java @@ -45,6 +45,8 @@ import io.netty.util.CharsetUtil; import java.net.InetSocketAddress; import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.SneakyThrows; @@ -87,7 +89,7 @@ public class DirectProxyHandler { private final Runnable onHandshakeCompleteAction; private final boolean tlsHostnameVerificationEnabled; final boolean tlsEnabledWithBroker; - private PulsarSslFactory sslFactory; + private Map pulsarSslFactoryMap; @SneakyThrows public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection) { @@ -102,27 +104,42 @@ public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection) this.tlsEnabledWithBroker = service.getConfiguration().isTlsEnabledWithBroker(); this.tlsHostnameVerificationEnabled = service.getConfiguration().isTlsHostnameVerificationEnabled(); this.onHandshakeCompleteAction = proxyConnection::cancelKeepAliveTask; - ProxyConfiguration config = service.getConfiguration(); - - if (tlsEnabledWithBroker) { - AuthenticationDataProvider authData = null; - - if (!isEmpty(config.getBrokerClientAuthenticationPlugin())) { - try { - authData = authentication.getAuthData(); - } catch (PulsarClientException e) { - throw new RuntimeException(e); - } - } - PulsarSslConfiguration sslConfiguration = buildSslConfiguration(config, authData); - this.sslFactory = (PulsarSslFactory) Class.forName(config.getSslFactoryPlugin()) - .getConstructor().newInstance(); - this.sslFactory.initialize(sslConfiguration); - this.sslFactory.createInternalSslContext(); - } + this.pulsarSslFactoryMap = new ConcurrentHashMap<>(); } public void connect(String brokerHostAndPort, InetSocketAddress targetBrokerAddress, int protocolVersion) { + String remoteHost; + try { + remoteHost = parseHost(brokerHostAndPort); + } catch (IllegalArgumentException e) { + log.warn("[{}] Failed to parse broker host '{}'", inboundChannel, brokerHostAndPort, e); + inboundChannel.close(); + return; + } + PulsarSslFactory sslFactory = + tlsEnabledWithBroker ? pulsarSslFactoryMap.computeIfAbsent(remoteHost, (hostname) -> { + AuthenticationDataProvider authData = null; + + if (!isEmpty(service.getConfiguration().getBrokerClientAuthenticationPlugin())) { + try { + authData = authentication.getAuthData(remoteHost); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + } + PulsarSslConfiguration sslConfiguration = + buildSslConfiguration(service.getConfiguration(), authData); + try { + PulsarSslFactory factory = + (PulsarSslFactory) Class.forName(service.getConfiguration().getSslFactoryPlugin()) + .getConstructor().newInstance(); + factory.initialize(sslConfiguration); + factory.createInternalSslContext(); + return factory; + } catch (Exception e) { + throw new RuntimeException(e); + } + }) : null; ProxyConfiguration config = service.getConfiguration(); // Start the connection attempt. @@ -142,15 +159,6 @@ public void connect(String brokerHostAndPort, InetSocketAddress targetBrokerAddr b.option(EpollChannelOption.EPOLL_MODE, EpollMode.LEVEL_TRIGGERED); } - String remoteHost; - try { - remoteHost = parseHost(brokerHostAndPort); - } catch (IllegalArgumentException e) { - log.warn("[{}] Failed to parse broker host '{}'", inboundChannel, brokerHostAndPort, e); - inboundChannel.close(); - return; - } - b.handler(new ChannelInitializer() { @Override protected void initChannel(SocketChannel ch) { 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 1148234be624c..ee8ae8d4afb3c 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 @@ -59,6 +59,7 @@ protected void setup() throws Exception { serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); + serviceStarter.getConfig().setBrokerWebServiceURLTLS(pulsar.getWebServiceAddressTls()); serviceStarter.getConfig().setBrokerClientTrustCertsFilePath(CA_CERT_FILE_PATH); serviceStarter.getConfig().setBrokerClientCertificateFilePath(BROKER_CERT_FILE_PATH); serviceStarter.getConfig().setBrokerClientKeyFilePath(BROKER_KEY_FILE_PATH); @@ -79,6 +80,7 @@ protected void setup() throws Exception { protected void doInitConf() throws Exception { super.doInitConf(); this.conf.setBrokerServicePortTls(Optional.of(0)); + this.conf.setWebServicePortTls(Optional.of(0)); this.conf.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); this.conf.setTlsKeyFilePath(PROXY_KEY_FILE_PATH); } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java index 4d73fd9f9b4e3..4fabf6d218549 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java @@ -251,7 +251,7 @@ public void runPerformanceTest() throws InterruptedException, IOException { Authentication auth = AuthenticationFactory.create(this.authPluginClassName, this.authParams); auth.start(); - AuthenticationDataProvider authData = auth.getAuthData(); + AuthenticationDataProvider authData = auth.getAuthData(produceUri.getHost()); if (authData.hasDataForHttp()) { for (Map.Entry kv : authData.getHttpHeaders()) { produceRequest.setHeader(kv.getKey(), kv.getValue()); From d377bc9d7321a66201a301b6887fb1fea3ef8820 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 3 Jan 2025 01:58:55 +0800 Subject: [PATCH 232/327] [improve][client] PIP-393: Improve performance of Negative Acknowledgement (#23600) Co-authored-by: Lari Hotari --- .../shell/src/assemble/LICENSE.bin.txt | 2 + pom.xml | 2 + .../pulsar/client/impl/NegativeAcksTest.java | 50 ++++++- pulsar-client-admin-shaded/pom.xml | 26 ++++ pulsar-client-all/pom.xml | 26 ++++ pulsar-client-dependencies-minimized/pom.xml | 100 +++++++++++++ pulsar-client-shaded/pom.xml | 26 ++++ pulsar-client/pom.xml | 10 ++ .../pulsar/client/impl/ConsumerImpl.java | 4 +- .../client/impl/NegativeAcksTracker.java | 139 ++++++++++++------ .../impl/conf/ConsumerConfigurationData.java | 10 ++ 11 files changed, 344 insertions(+), 51 deletions(-) create mode 100644 pulsar-client-dependencies-minimized/pom.xml diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 05342d1724399..3333c9fe6ab66 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -418,6 +418,8 @@ The Apache Software License, Version 2.0 - avro-protobuf-1.11.4.jar * RE2j -- re2j-1.7.jar * Spotify completable-futures -- completable-futures-0.3.6.jar + * RoaringBitmap -- RoaringBitmap-1.2.0.jar + * Fastutil -- fastutil-8.5.14.jar BSD 3-clause "New" or "Revised" License * JSR305 -- jsr305-3.0.2.jar -- ../licenses/LICENSE-JSR305.txt diff --git a/pom.xml b/pom.xml index 93cd3d5e11f15..3cd9bd4b8d44d 100644 --- a/pom.xml +++ b/pom.xml @@ -2580,6 +2580,7 @@ flexible messaging model and an intuitive client API. pulsar-metadata jetcd-core-shaded jclouds-shaded + pulsar-client-dependencies-minimized pulsar-package-management @@ -2645,6 +2646,7 @@ flexible messaging model and an intuitive client API. distribution pulsar-metadata jetcd-core-shaded + pulsar-client-dependencies-minimized pulsar-package-management 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 b372ecabc5de4..f8bc30f09667c 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 @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import java.util.HashSet; @@ -311,7 +312,7 @@ public void testNegativeAcksDeleteFromUnackedTracker() throws Exception { // negative topic message id consumer.negativeAcknowledge(topicMessageId); NegativeAcksTracker negativeAcksTracker = consumer.getNegativeAcksTracker(); - assertEquals(negativeAcksTracker.getNackedMessagesCount().orElse((long) -1).longValue(), 1L); + assertEquals(negativeAcksTracker.getNackedMessagesCount(), 1L); assertEquals(unAckedMessageTracker.size(), 0); negativeAcksTracker.close(); // negative batch message id @@ -319,11 +320,56 @@ public void testNegativeAcksDeleteFromUnackedTracker() throws Exception { consumer.negativeAcknowledge(batchMessageId); consumer.negativeAcknowledge(batchMessageId2); consumer.negativeAcknowledge(batchMessageId3); - assertEquals(negativeAcksTracker.getNackedMessagesCount().orElse((long) -1).longValue(), 1L); + assertEquals(negativeAcksTracker.getNackedMessagesCount(), 1L); assertEquals(unAckedMessageTracker.size(), 0); negativeAcksTracker.close(); } + /** + * If we nack multiple messages in the same batch with different redelivery delays, the messages should be redelivered + * with the correct delay. However, all messages are redelivered at the same time. + * @throws Exception + */ + @Test + public void testNegativeAcksWithBatch() throws Exception { + cleanup(); + conf.setAcknowledgmentAtBatchIndexLevelEnabled(true); + setup(); + String topic = BrokerTestUtil.newUniqueName("testNegativeAcksWithBatch"); + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub1") + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .enableBatchIndexAcknowledgment(true) + .negativeAckRedeliveryDelay(3, TimeUnit.SECONDS) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .enableBatching(true) + .batchingMaxPublishDelay(1, TimeUnit.HOURS) + .batchingMaxMessages(2) + .create(); + // send two messages in the same batch + producer.sendAsync("test-0"); + producer.sendAsync("test-1"); + producer.flush(); + + // negative ack the first message + consumer.negativeAcknowledge(consumer.receive()); + // wait for 2s, negative ack the second message + Thread.sleep(2000); + consumer.negativeAcknowledge(consumer.receive()); + + // now 2s has passed, the first message should be redelivered 1s later. + Message msg1 = consumer.receive(2, TimeUnit.SECONDS); + assertNotNull(msg1); + } + @Test public void testNegativeAcksWithBatchAckEnabled() throws Exception { cleanup(); diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index f667a8eb61e3f..de54c3d049639 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -34,6 +34,17 @@ ${project.groupId} pulsar-client-admin-original ${project.version} + + + it.unimi.dsi + fastutil + + + + + ${project.groupId} + pulsar-client-dependencies-minimized + ${project.version} ${project.groupId} @@ -150,6 +161,8 @@ org.objenesis:* org.reactivestreams:reactive-streams org.yaml:snakeyaml + org.apache.pulsar:pulsar-client-dependencies-minimized + org.roaringbitmap:RoaringBitmap com.fasterxml.jackson.core:jackson-annotations @@ -269,6 +282,10 @@ io.swagger org.apache.pulsar.shade.io.swagger + + it.unimi.dsi.fastutil + org.apache.pulsar.shade.it.unimi.dsi.fastutil + javassist org.apache.pulsar.shade.javassist @@ -313,6 +330,11 @@ META-INF/versions/$1/org/apache/pulsar/shade/org/glassfish/ true + + META-INF/versions/(\d+)/org/roaringbitmap/ + META-INF/versions/$1/org/apache/pulsar/shade/org/roaringbitmap/ + true + META-INF/versions/(\d+)/org/yaml/ META-INF/versions/$1/org/apache/pulsar/shade/org/yaml/ @@ -374,6 +396,10 @@ org.reactivestreams org.apache.pulsar.shade.org.reactivestreams + + org.roaringbitmap + org.apache.pulsar.shade.org.roaringbitmap + org.yaml org.apache.pulsar.shade.org.yaml diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 5e30dbd999de4..4fec9ff51b894 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -39,6 +39,17 @@ ${project.groupId} pulsar-client-original ${project.version} + + + it.unimi.dsi + fastutil + + + + + ${project.groupId} + pulsar-client-dependencies-minimized + ${project.version} ${project.groupId} @@ -200,6 +211,8 @@ org.reactivestreams:reactive-streams org.tukaani:xz org.yaml:snakeyaml + org.apache.pulsar:pulsar-client-dependencies-minimized + org.roaringbitmap:RoaringBitmap com.fasterxml.jackson.core:jackson-annotations @@ -317,6 +330,10 @@ io.swagger org.apache.pulsar.shade.io.swagger + + it.unimi.dsi.fastutil + org.apache.pulsar.shade.it.unimi.dsi.fastutil + javassist org.apache.pulsar.shade.javassist @@ -361,6 +378,11 @@ META-INF/versions/$1/org/apache/pulsar/shade/org/glassfish/ true + + META-INF/versions/(\d+)/org/roaringbitmap/ + META-INF/versions/$1/org/apache/pulsar/shade/org/roaringbitmap/ + true + META-INF/versions/(\d+)/org/yaml/ META-INF/versions/$1/org/apache/pulsar/shade/org/yaml/ @@ -439,6 +461,10 @@ org.reactivestreams org.apache.pulsar.shade.org.reactivestreams + + org.roaringbitmap + org.apache.pulsar.shade.org.roaringbitmap + org.tukaani org.apache.pulsar.shade.org.tukaani diff --git a/pulsar-client-dependencies-minimized/pom.xml b/pulsar-client-dependencies-minimized/pom.xml new file mode 100644 index 0000000000000..e838fedfddca5 --- /dev/null +++ b/pulsar-client-dependencies-minimized/pom.xml @@ -0,0 +1,100 @@ + + + + 4.0.0 + + org.apache.pulsar + pulsar + 4.1.0-SNAPSHOT + + + pulsar-client-dependencies-minimized + Apache Pulsar :: Client :: Dependencies minimized + This module is used in `pulsar-client-all`, `pulsar-client-shaded`, and `pulsar-client-admin-shaded` + to minimize the number of classes included in the shaded jars for specific dependencies. + Currently, it is used to minimize the classes included from `fastutil`. + + + + ${project.groupId} + pulsar-client-original + ${project.version} + + + + ${project.artifactId}-${project.version} + + + org.apache.maven.plugins + maven-deploy-plugin + + + true + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + true + false + + true + + + + org.apache.pulsar:pulsar-client-original + + it.unimi.dsi:fastutil + + + + + + org.apache.pulsar:pulsar-client-original + + ** + + + ** + + + + + + + + + + diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index 62bab3cb2d71b..d8adacbe8a0bb 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -39,6 +39,17 @@ ${project.groupId} pulsar-client-original ${project.version} + + + it.unimi.dsi + fastutil + + + + + ${project.groupId} + pulsar-client-dependencies-minimized + ${project.version} ${project.groupId} @@ -164,6 +175,8 @@ org.reactivestreams:reactive-streams org.tukaani:xz org.yaml:snakeyaml + org.apache.pulsar:pulsar-client-dependencies-minimized + org.roaringbitmap:RoaringBitmap com.fasterxml.jackson.core:jackson-annotations @@ -263,6 +276,10 @@ io.swagger org.apache.pulsar.shade.io.swagger + + it.unimi.dsi.fastutil + org.apache.pulsar.shade.it.unimi.dsi.fastutil + javax.activation org.apache.pulsar.shade.javax.activation @@ -281,6 +298,11 @@ true + + META-INF/versions/(\d+)/org/roaringbitmap/ + META-INF/versions/$1/org/apache/pulsar/shade/org/roaringbitmap/ + true + META-INF/versions/(\d+)/org/yaml/ META-INF/versions/$1/org/apache/pulsar/shade/org/yaml/ @@ -343,6 +365,10 @@ org.reactivestreams org.apache.pulsar.shade.org.reactivestreams + + org.roaringbitmap + org.apache.pulsar.shade.org.roaringbitmap + org.tukaani org.apache.pulsar.shade.org.tukaani diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 49bb3c6490ae9..e1a70ed074833 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -207,6 +207,16 @@ test + + org.roaringbitmap + RoaringBitmap + + + + it.unimi.dsi + fastutil + + 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 16dc70f736e7d..86af4bdaf58c8 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 @@ -2752,7 +2752,7 @@ private int removeExpiredMessagesFromQueue(Set messageIds) { int messagesFromQueue = 0; Message peek = incomingMessages.peek(); if (peek != null) { - MessageIdAdv messageId = MessageIdAdvUtils.discardBatch(peek.getMessageId()); + MessageId messageId = NegativeAcksTracker.discardBatchAndPartitionIndex(peek.getMessageId()); if (!messageIds.contains(messageId)) { // first message is not expired, then no message is expired in queue. return 0; @@ -2763,7 +2763,7 @@ private int removeExpiredMessagesFromQueue(Set messageIds) { while (message != null) { decreaseIncomingMessageSize(message); messagesFromQueue++; - MessageIdAdv id = MessageIdAdvUtils.discardBatch(message.getMessageId()); + MessageId id = NegativeAcksTracker.discardBatchAndPartitionIndex(message.getMessageId()); if (!messageIds.contains(id)) { messageIds.add(id); break; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java index 5256ebf04f43c..273880569c307 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java @@ -22,9 +22,13 @@ import com.google.common.annotations.VisibleForTesting; import io.netty.util.Timeout; import io.netty.util.Timer; +import it.unimi.dsi.fastutil.longs.Long2ObjectAVLTreeMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectSortedMap; +import it.unimi.dsi.fastutil.longs.LongBidirectionalIterator; import java.io.Closeable; import java.util.HashSet; -import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.Message; @@ -32,40 +36,37 @@ import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.RedeliveryBackoff; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; -import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; +import org.roaringbitmap.longlong.Roaring64Bitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; class NegativeAcksTracker implements Closeable { private static final Logger log = LoggerFactory.getLogger(NegativeAcksTracker.class); - private ConcurrentLongLongPairHashMap nackedMessages = null; + // timestamp -> ledgerId -> entryId, no need to batch index, if different messages have + // different timestamp, there will be multiple entries in the map + // RB Tree -> LongOpenHashMap -> Roaring64Bitmap + private Long2ObjectSortedMap> nackedMessages = null; private final ConsumerBase consumer; private final Timer timer; - private final long nackDelayNanos; - private final long timerIntervalNanos; + private final long nackDelayMs; private final RedeliveryBackoff negativeAckRedeliveryBackoff; + private final int negativeAckPrecisionBitCnt; private Timeout timeout; // Set a min delay to allow for grouping nacks within a single batch - private static final long MIN_NACK_DELAY_NANOS = TimeUnit.MILLISECONDS.toNanos(100); - private static final long NON_PARTITIONED_TOPIC_PARTITION_INDEX = Long.MAX_VALUE; + private static final long MIN_NACK_DELAY_MS = 100; + private static final int DUMMY_PARTITION_INDEX = -2; public NegativeAcksTracker(ConsumerBase consumer, ConsumerConfigurationData conf) { this.consumer = consumer; this.timer = consumer.getClient().timer(); - this.nackDelayNanos = Math.max(TimeUnit.MICROSECONDS.toNanos(conf.getNegativeAckRedeliveryDelayMicros()), - MIN_NACK_DELAY_NANOS); + this.nackDelayMs = Math.max(TimeUnit.MICROSECONDS.toMillis(conf.getNegativeAckRedeliveryDelayMicros()), + MIN_NACK_DELAY_MS); this.negativeAckRedeliveryBackoff = conf.getNegativeAckRedeliveryBackoff(); - if (negativeAckRedeliveryBackoff != null) { - this.timerIntervalNanos = Math.max( - TimeUnit.MILLISECONDS.toNanos(negativeAckRedeliveryBackoff.next(0)), - MIN_NACK_DELAY_NANOS) / 3; - } else { - this.timerIntervalNanos = nackDelayNanos / 3; - } + this.negativeAckPrecisionBitCnt = conf.getNegativeAckPrecisionBitCnt(); } private void triggerRedelivery(Timeout t) { @@ -76,21 +77,48 @@ private void triggerRedelivery(Timeout t) { return; } - long now = System.nanoTime(); - nackedMessages.forEach((ledgerId, entryId, partitionIndex, timestamp) -> { - if (timestamp < now) { - MessageId msgId = new MessageIdImpl(ledgerId, entryId, - // need to covert non-partitioned topic partition index to -1 - (int) (partitionIndex == NON_PARTITIONED_TOPIC_PARTITION_INDEX ? -1 : partitionIndex)); - addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); - messagesToRedeliver.add(msgId); + long currentTimestamp = System.currentTimeMillis(); + for (long timestamp : nackedMessages.keySet()) { + if (timestamp > currentTimestamp) { + // We are done with all the messages that need to be redelivered + break; + } + + Long2ObjectMap ledgerMap = nackedMessages.get(timestamp); + for (Long2ObjectMap.Entry ledgerEntry : ledgerMap.long2ObjectEntrySet()) { + long ledgerId = ledgerEntry.getLongKey(); + Roaring64Bitmap entrySet = ledgerEntry.getValue(); + entrySet.forEach(entryId -> { + MessageId msgId = new MessageIdImpl(ledgerId, entryId, DUMMY_PARTITION_INDEX); + addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); + messagesToRedeliver.add(msgId); + }); + } + } + + // remove entries from the nackedMessages map + LongBidirectionalIterator iterator = nackedMessages.keySet().iterator(); + while (iterator.hasNext()) { + long timestamp = iterator.nextLong(); + if (timestamp <= currentTimestamp) { + iterator.remove(); + } else { + break; } - }); - for (MessageId messageId : messagesToRedeliver) { - nackedMessages.remove(((MessageIdImpl) messageId).getLedgerId(), - ((MessageIdImpl) messageId).getEntryId()); } - this.timeout = timer.newTimeout(this::triggerRedelivery, timerIntervalNanos, TimeUnit.NANOSECONDS); + + // Schedule the next redelivery if there are still messages to redeliver + if (!nackedMessages.isEmpty()) { + long nextTriggerTimestamp = nackedMessages.firstLongKey(); + long delayMs = Math.max(nextTriggerTimestamp - currentTimestamp, 0); + if (delayMs > 0) { + this.timeout = timer.newTimeout(this::triggerRedelivery, delayMs, TimeUnit.MILLISECONDS); + } else { + this.timeout = timer.newTimeout(this::triggerRedelivery, 0, TimeUnit.MILLISECONDS); + } + } else { + this.timeout = null; + } } // release the lock of NegativeAcksTracker before calling consumer.redeliverUnacknowledgedMessages, @@ -110,39 +138,56 @@ public synchronized void add(Message message) { add(message.getMessageId(), message.getRedeliveryCount()); } + static long trimLowerBit(long timestamp, int bits) { + return timestamp & (-1L << bits); + } + private synchronized void add(MessageId messageId, int redeliveryCount) { if (nackedMessages == null) { - nackedMessages = ConcurrentLongLongPairHashMap.newBuilder() - .autoShrink(true) - .concurrencyLevel(1) - .build(); + nackedMessages = new Long2ObjectAVLTreeMap<>(); } - long backoffNs; + long backoffMs; if (negativeAckRedeliveryBackoff != null) { - backoffNs = TimeUnit.MILLISECONDS.toNanos(negativeAckRedeliveryBackoff.next(redeliveryCount)); + backoffMs = TimeUnit.MILLISECONDS.toMillis(negativeAckRedeliveryBackoff.next(redeliveryCount)); } else { - backoffNs = nackDelayNanos; + backoffMs = nackDelayMs; } - MessageIdAdv messageIdAdv = MessageIdAdvUtils.discardBatch(messageId); - // ConcurrentLongLongPairHashMap requires the key and value >=0. - // partitionIndex is -1 if the message is from a non-partitioned topic, but we don't use - // partitionIndex actually, so we can set it to Long.MAX_VALUE in the case of non-partitioned topic to - // avoid exception from ConcurrentLongLongPairHashMap. - nackedMessages.put(messageIdAdv.getLedgerId(), messageIdAdv.getEntryId(), - messageIdAdv.getPartitionIndex() >= 0 ? messageIdAdv.getPartitionIndex() : - NON_PARTITIONED_TOPIC_PARTITION_INDEX, System.nanoTime() + backoffNs); + MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; + long timestamp = trimLowerBit(System.currentTimeMillis() + backoffMs, negativeAckPrecisionBitCnt); + nackedMessages.computeIfAbsent(timestamp, k -> new Long2ObjectOpenHashMap<>()) + .computeIfAbsent(messageIdAdv.getLedgerId(), k -> new Roaring64Bitmap()) + .add(messageIdAdv.getEntryId()); if (this.timeout == null) { // Schedule a task and group all the redeliveries for same period. Leave a small buffer to allow for // nack immediately following the current one will be batched into the same redeliver request. - this.timeout = timer.newTimeout(this::triggerRedelivery, timerIntervalNanos, TimeUnit.NANOSECONDS); + this.timeout = timer.newTimeout(this::triggerRedelivery, backoffMs, TimeUnit.MILLISECONDS); } } + /** + * Discard the batch index and partition index from the message id. + * + * @param messageId + * @return + */ + public static MessageIdAdv discardBatchAndPartitionIndex(MessageId messageId) { + if (messageId instanceof ChunkMessageIdImpl) { + return (MessageIdAdv) messageId; + } + MessageIdAdv msgId = (MessageIdAdv) messageId; + return new MessageIdImpl(msgId.getLedgerId(), msgId.getEntryId(), DUMMY_PARTITION_INDEX); + } + @VisibleForTesting - Optional getNackedMessagesCount() { - return Optional.ofNullable(nackedMessages).map(ConcurrentLongLongPairHashMap::size); + synchronized long getNackedMessagesCount() { + if (nackedMessages == null) { + return 0; + } + return nackedMessages.values().stream().mapToLong( + ledgerMap -> ledgerMap.values().stream().mapToLong( + Roaring64Bitmap::getLongCardinality).sum()).sum(); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index cd82b54618f22..dc9251a975c39 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -156,6 +156,16 @@ public class ConsumerConfigurationData implements Serializable, Cloneable { ) private long negativeAckRedeliveryDelayMicros = TimeUnit.MINUTES.toMicros(1); + @ApiModelProperty( + name = "negativeAckPrecisionBitCnt", + value = "The redelivery time precision bit count. The lower bits of the redelivery time will be" + + "trimmed to reduce the memory occupation.\nThe default value is 8, which means the" + + "redelivery time will be bucketed by 256ms, the redelivery time could be earlier(no later)" + + "than the expected time, but no more than 256ms. \nIf set to k, the redelivery time will be" + + "bucketed by 2^k ms.\nIf the value is 0, the redelivery time will be accurate to ms." + ) + private int negativeAckPrecisionBitCnt = 8; + @ApiModelProperty( name = "maxTotalReceiverQueueSizeAcrossPartitions", value = "The max total receiver queue size across partitions.\n" From 3d71c8794e5ccc560c431d2066a442d5816e3464 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 3 Jan 2025 16:11:42 +0800 Subject: [PATCH 233/327] [fix] [broker] Fix items in dispatcher.recentlyJoinedConsumers are out-of-order, which may cause a delivery stuck (#23802) --- ...KeyDispatcherMultipleConsumersClassic.java | 37 ++- ...ntryCacheKeySharedSubscriptionV30Test.java | 289 ++++++++++++++++++ 2 files changed, 318 insertions(+), 8 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/NonEntryCacheKeySharedSubscriptionV30Test.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java index 71f37c5939d6a..56161d8dd1544 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java @@ -34,6 +34,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import lombok.Setter; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; @@ -82,6 +83,15 @@ public class PersistentStickyKeyDispatcherMultipleConsumersClassic */ private final LinkedHashMap recentlyJoinedConsumers; + /** + * The method {@link #sortRecentlyJoinedConsumersIfNeeded} is a un-normal method, which used to fix the issue that + * was described at https://github.com/apache/pulsar/pull/23795. + * To cover the case that does not contain the hot fix that https://github.com/apache/pulsar/pull/23795 provided, + * we add this method to reproduce the issue in tests. + **/ + @Setter + public boolean sortRecentlyJoinedConsumersIfNeeded = true; + PersistentStickyKeyDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, Subscription subscription, ServiceConfiguration conf, KeySharedMeta ksm) { @@ -144,7 +154,11 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { if (!allowOutOfOrderDelivery && recentlyJoinedConsumers != null && consumerList.size() > 1 - && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) { + && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1 + // If there is a delayed "cursor.rewind" after the pending read, the consumers that will be + // added before the "cursor.rewind" will have a same "recent joined position", which is the + // same as "mark deleted position +1", so we can skip this adding. + && !shouldRewindBeforeReadingOrReplaying) { recentlyJoinedConsumers.put(consumer, readPositionWhenJoining); sortRecentlyJoinedConsumersIfNeeded(); } @@ -153,9 +167,13 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { } private void sortRecentlyJoinedConsumersIfNeeded() { + if (!sortRecentlyJoinedConsumersIfNeeded) { + return; + } if (recentlyJoinedConsumers.size() == 1) { return; } + // Since we check the order of queue after each consumer joined, we can only check the last two items. boolean sortNeeded = false; Position posPre = null; Position posAfter = null; @@ -163,18 +181,21 @@ private void sortRecentlyJoinedConsumersIfNeeded() { if (posPre == null) { posPre = entry.getValue(); } else { + posPre = posAfter; posAfter = entry.getValue(); } - if (posPre != null && posAfter != null) { - if (posPre.compareTo(posAfter) > 0) { - sortNeeded = true; - break; - } - posPre = posAfter; + } + if (posPre != null && posAfter != null) { + if (posPre.compareTo(posAfter) > 0) { + sortNeeded = true; } } - + // Something went wrongly, sort the collection. if (sortNeeded) { + log.error("[{}] [{}] The items in recentlyJoinedConsumers are out-of-order. {}", + topic.getName(), name, recentlyJoinedConsumers.entrySet().stream().map(entry -> + String.format("%s-%s:%s", entry.getKey().consumerName(), entry.getValue().getLedgerId(), + entry.getValue().getEntryId())).collect(Collectors.toList())); List> sortedList = new ArrayList<>(recentlyJoinedConsumers.entrySet()); Collections.sort(sortedList, Map.Entry.comparingByValue()); recentlyJoinedConsumers.clear(); diff --git a/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/NonEntryCacheKeySharedSubscriptionV30Test.java b/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/NonEntryCacheKeySharedSubscriptionV30Test.java new file mode 100644 index 0000000000000..0de3d4edfad64 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/NonEntryCacheKeySharedSubscriptionV30Test.java @@ -0,0 +1,289 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; +import static org.testng.Assert.assertTrue; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.mledger.Position; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumersClassic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.awaitility.Awaitility; +import org.mockito.stubbing.Answer; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class NonEntryCacheKeySharedSubscriptionV30Test extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setManagedLedgerCacheSizeMB(0); + this.conf.setManagedLedgerMaxEntriesPerLedger(50000); + // Use the implementation of subscriptions in v3.x. + this.conf.setSubscriptionKeySharedUseClassicPersistentImplementation(true); + } + + + @Test(timeOut = 180 * 1000, invocationCount = 1) + public void testRecentJoinQueueIsInOrderAfterRewind() throws Exception { + int msgCount = 300; + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subName = "my-sub"; + final DefaultThreadFactory threadFactory = + new DefaultThreadFactory(BrokerTestUtil.newUniqueName("thread")); + admin.topics().createNonPartitionedTopic(topic); + admin.topics().createSubscription(topic, subName, MessageId.earliest); + + // Send messages. + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32).topic(topic).enableBatching(false).create(); + AtomicInteger msgGenerator = new AtomicInteger(); + for (int i = 0; i < msgCount; i++) { + int v = msgGenerator.getAndIncrement(); + producer.newMessage().key(String.valueOf(v)).value(v).send(); + } + + // Make ack holes. + Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(100) + .subscriptionType(SubscriptionType.Key_Shared) + .consumerName("c1") + .subscribe(); + Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(100) + .subscriptionType(SubscriptionType.Key_Shared) + .consumerName("c2") + .subscribe(); + Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(100) + .subscriptionType(SubscriptionType.Key_Shared) + .consumerName("c3") + .subscribe(); + final PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + final PersistentSubscription persistentSubscription = persistentTopic.getSubscription(subName); + final PersistentStickyKeyDispatcherMultipleConsumersClassic dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumersClassic) persistentSubscription.getDispatcher(); + final ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(subName); + dispatcher.setSortRecentlyJoinedConsumersIfNeeded(false); + + // Make ack holes. + // - ack all messages that consumer1 or consumer2 received. + // - do not ack messages that consumer2 received. + ackAllMessages(consumer1, consumer2); + Position mdPosition = (Position) cursor.getMarkDeletedPosition(); + Position readPosition = (Position) cursor.getReadPosition(); + Position LAC = (Position) ml.getLastConfirmedEntry(); + assertTrue(readPosition.compareTo(LAC) >= 0); + Position firstWaitingAckPos = ml.getNextValidPosition(mdPosition); + log.info("md-pos {}:{}", mdPosition.getLedgerId(), mdPosition.getEntryId()); + log.info("rd-pos {}:{}", readPosition.getLedgerId(), readPosition.getEntryId()); + log.info("lac-pos {}:{}", LAC.getLedgerId(), LAC.getEntryId()); + log.info("first-waiting-ack-pos {}:{}", firstWaitingAckPos.getLedgerId(), firstWaitingAckPos.getEntryId()); + + // Inject a delay for the next replay read. + LedgerHandle firstLedger = ml.currentLedger; + Assert.assertEquals(firstWaitingAckPos.getLedgerId(), firstLedger.getId()); + LedgerHandle spyFirstLedger = spy(firstLedger); + CountDownLatch replyReadSignal = new CountDownLatch(1); + AtomicBoolean replayReadWasTriggered = new AtomicBoolean(); + Answer answer = invocation -> { + long firstEntry = (long) invocation.getArguments()[0]; + if (firstEntry == firstWaitingAckPos.getEntryId()) { + replayReadWasTriggered.set(true); + final CompletableFuture res = new CompletableFuture<>(); + threadFactory.newThread(() -> { + try { + replyReadSignal.await(); + CompletableFuture future = + (CompletableFuture) invocation.callRealMethod(); + future.thenAccept(v -> { + res.complete(v); + }).exceptionally(ex -> { + res.completeExceptionally(ex); + return null; + }); + } catch (Throwable ex) { + res.completeExceptionally(ex); + } + }).start(); + return res; + } else { + return invocation.callRealMethod(); + } + }; + doAnswer(answer).when(spyFirstLedger).readAsync(anyLong(), anyLong()); + doAnswer(answer).when(spyFirstLedger).readUnconfirmedAsync(anyLong(), anyLong()); + ml.currentLedger = spyFirstLedger; + + // Keep publish to avoid pending normal read. + AtomicBoolean keepPublishing = new AtomicBoolean(true); + new Thread(() -> { + while (keepPublishing.get()) { + int v = msgGenerator.getAndIncrement(); + producer.newMessage().key(String.valueOf(v)).value(v).sendAsync(); + sleep(100); + } + }).start(); + + // Trigger a message redelivery. + consumer3.close(); + Awaitility.await().until(() -> replayReadWasTriggered.get()); + + // Close all consumers to trigger a cursor.rewind. + consumer1.close(); + consumer2.close(); + + // Start 100 consumers. + List>> consumerList = new ArrayList<>(); + for (int i = 0; i < 40; i++) { + consumerList.add(pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribeAsync()); + if (i == 10) { + for (int j = 0; j < msgCount; j++) { + int v = msgGenerator.getAndIncrement(); + producer.newMessage().key(String.valueOf(v)).value(v).send(); + } + final Consumer firstConsumer = consumerList.get(0).join(); + ackAllMessages(firstConsumer); + new Thread(() -> { + while (keepPublishing.get()) { + try { + ackAllMessages(firstConsumer); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }).start(); + } + log.info("recent-joined-consumers {} {}", i, dispatcher.getRecentlyJoinedConsumers().size()); + if (dispatcher.getRecentlyJoinedConsumers().size() > 0) { + Position mdPosition2 = (Position) cursor.getMarkDeletedPosition(); + Position readPosition2 = (Position) cursor.getReadPosition(); + Position LAC2 = (Position) ml.getLastConfirmedEntry(); + assertTrue(readPosition.compareTo(LAC) >= 0); + Position firstWaitingAckPos2 = ml.getNextValidPosition(mdPosition); + if(readPosition2.compareTo(firstWaitingAckPos) > 0) { + keepPublishing.set(false); + log.info("consumer-index: {}", i); + log.info("md-pos-2 {}:{}", mdPosition2.getLedgerId(), mdPosition2.getEntryId()); + log.info("rd-pos-2 {}:{}", readPosition2.getLedgerId(), readPosition2.getEntryId()); + log.info("lac-pos-2 {}:{}", LAC2.getLedgerId(), LAC2.getEntryId()); + log.info("first-waiting-ack-pos-2 {}:{}", firstWaitingAckPos2.getLedgerId(), + firstWaitingAckPos2.getEntryId()); + // finish the replay read here. + replyReadSignal.countDown(); + } else { + sleep(1000); + } + } + } + consumerList.get(consumerList.size() - 1).join(); + + synchronized (dispatcher) { + LinkedHashMap recentJoinedConsumers = dispatcher.getRecentlyJoinedConsumers(); + assertTrue(verifyMapItemsAreInOrder(recentJoinedConsumers)); + } + + // cleanup. + producer.close(); + for (CompletableFuture> c : consumerList) { + c.join().close(); + } + admin.topics().delete(topic, false); + } + + private void sleep(int millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + private boolean verifyMapItemsAreInOrder(LinkedHashMap map) { + boolean outOfOrder = false; + Position posPre = null; + Position posAfter = null; + for (Map.Entry entry : map.entrySet()) { + if (posPre == null) { + posPre = (Position) entry.getValue(); + } else { + posAfter = (Position) entry.getValue(); + } + if (posPre != null && posAfter != null) { + if (posPre.compareTo(posAfter) > 0) { + outOfOrder = true; + break; + } + posPre = posAfter; + } + } + return !outOfOrder; + } +} From a6986b1b664bf2a8da6e99c2db39c65b08b9f3a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Fri, 3 Jan 2025 16:35:39 +0800 Subject: [PATCH 234/327] [improve][broker] Improve SystemTopicBasedTopicPoliciesService reader to reduce GC pressure (#23780) --- .../SystemTopicBasedTopicPoliciesService.java | 14 +++++++++++--- .../systopic/TopicPoliciesSystemTopicClient.java | 1 + .../broker/service/TopicPolicyTestUtils.java | 5 ++++- .../NamespaceEventsSystemTopicServiceTest.java | 2 ++ .../systopic/PartitionedSystemTopicTest.java | 6 +++++- 5 files changed, 23 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 5488d5563f607..f2206f5cada89 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -428,7 +428,11 @@ private void initPolicesCache(SystemTopicClient.Reader reader, Comp } if (hasMore) { reader.readNextAsync().thenAccept(msg -> { - refreshTopicPoliciesCache(msg); + try { + refreshTopicPoliciesCache(msg); + } finally { + msg.release(); + } if (log.isDebugEnabled()) { log.debug("[{}] Loop next event reading for system topic.", reader.getSystemTopic().getTopicName().getNamespaceObject()); @@ -505,8 +509,12 @@ private void readMorePoliciesAsync(SystemTopicClient.Reader reader) } reader.readNextAsync() .thenAccept(msg -> { - refreshTopicPoliciesCache(msg); - notifyListener(msg); + try { + refreshTopicPoliciesCache(msg); + notifyListener(msg); + } finally { + msg.release(); + } }) .whenComplete((__, ex) -> { if (ex == null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java index ea3ac507d1128..6ada9db2967c7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java @@ -73,6 +73,7 @@ protected CompletableFuture> newReaderAsyncInternal() { .subscriptionRolePrefix(SystemTopicNames.SYSTEM_READER_PREFIX) .startMessageId(MessageId.earliest) .readCompacted(true) + .poolMessages(true) .createAsync() .thenApply(reader -> { if (log.isDebugEnabled()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java index 9cf688d62edc6..d4275cdfd200f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java @@ -21,6 +21,7 @@ import java.util.Optional; import java.util.concurrent.ExecutionException; import lombok.Cleanup; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.common.events.PulsarEvent; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; @@ -67,7 +68,9 @@ public static Optional getTopicPoliciesBypassCache(TopicPoliciesS .newReader(); PulsarEvent event = null; while (reader.hasMoreEvents()) { - event = reader.readNext().getValue(); + @Cleanup("release") + Message message = reader.readNext(); + event = message.getValue(); } return Optional.ofNullable(event).map(e -> e.getTopicPoliciesEvent().getPolicies()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java index e66140efb32bb..aaa719515c99f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java @@ -129,6 +129,7 @@ public void testSendAndReceiveNamespaceEvents() throws Exception { .build(); systemTopicClientForNamespace1.newWriter().write(getEventKey(event), event); SystemTopicClient.Reader reader = systemTopicClientForNamespace1.newReader(); + @Cleanup("release") Message received = reader.readNext(); log.info("Receive pulsar event from system topic : {}", received.getValue()); @@ -139,6 +140,7 @@ public void testSendAndReceiveNamespaceEvents() throws Exception { // test new reader read SystemTopicClient.Reader reader1 = systemTopicClientForNamespace1.newReader(); + @Cleanup("release") Message received1 = reader1.readNext(); log.info("Receive pulsar event from system topic : {}", received1.getValue()); Assert.assertEquals(received1.getValue(), event); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/PartitionedSystemTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/PartitionedSystemTopicTest.java index e7bfa3278e36d..e31f78665b394 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/PartitionedSystemTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/PartitionedSystemTopicTest.java @@ -338,9 +338,13 @@ public void testSystemTopicNotCheckExceed() throws Exception { FutureUtil.waitForAll(List.of(writer1, writer2, f1)).join(); Assert.assertTrue(reader1.hasMoreEvents()); - Assert.assertNotNull(reader1.readNext()); + Message message = reader1.readNext(); + Assert.assertNotNull(message); + message.release(); Assert.assertTrue(reader2.hasMoreEvents()); + message = reader2.readNext(); Assert.assertNotNull(reader2.readNext()); + message.release(); reader1.close(); reader2.close(); writer1.get().close(); From 4a93e2503bf1d890f5d45f97374d37ee575372d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Fri, 3 Jan 2025 16:39:55 +0800 Subject: [PATCH 235/327] [improve][txn] Improve Reader in TransactionBuffer to reduce GC pressure (#23779) --- ...sactionBufferSnapshotBaseSystemTopicClient.java | 1 + .../SnapshotSegmentAbortedTxnProcessorImpl.java | 12 ++++++++++-- .../broker/transaction/buffer/impl/TableView.java | 14 +++++++++----- .../SegmentAbortedTxnProcessorTest.java | 2 ++ .../TopicTransactionBufferRecoverTest.java | 4 +++- 5 files changed, 25 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index 4023cd88bef55..7ba01b09b2790 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -205,6 +205,7 @@ protected CompletableFuture> newReaderAsyncInternal() { .subscriptionRolePrefix(SystemTopicNames.SYSTEM_READER_PREFIX) .startMessageId(MessageId.earliest) .readCompacted(true) + .poolMessages(true) .createAsync() .thenApply(reader -> { if (log.isDebugEnabled()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java index f2ff5d519d8c0..779d083289b70 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java @@ -768,8 +768,16 @@ private CompletableFuture clearAllSnapshotSegments() { try { while (wait(reader.hasMoreEventsAsync(), "has more events")) { final var message = wait(reader.readNextAsync(), "read next"); - if (topic.getName().equals(message.getValue().getTopicName())) { - snapshotSegmentsWriter.getFuture().get().write(message.getKey(), null); + final String topicName; + final String key; + try { + topicName = message.getValue().getTopicName(); + key = message.getKey(); + } finally { + message.release(); + } + if (topic.getName().equals(topicName)) { + snapshotSegmentsWriter.getFuture().get().write(key, null); } } future.complete(null); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TableView.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TableView.java index 7608a393cc980..40adec7488420 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TableView.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TableView.java @@ -61,12 +61,16 @@ public T readLatest(String topic) throws Exception { final var reader = getReader(topic); while (wait(reader.hasMoreEventsAsync(), "has more events")) { final var msg = wait(reader.readNextAsync(), "read message"); - if (msg.getKey() != null) { - if (msg.getValue() != null) { - snapshots.put(msg.getKey(), msg.getValue()); - } else { - snapshots.remove(msg.getKey()); + try { + if (msg.getKey() != null) { + if (msg.getValue() != null) { + snapshots.put(msg.getKey(), msg.getValue()); + } else { + snapshots.remove(msg.getKey()); + } } + } finally { + msg.release(); } } return snapshots.get(topic); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java index d9ba825f02e93..b9f4c4f632cc0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java @@ -348,6 +348,7 @@ private void verifySnapshotSegmentsSize(String topic, int size) throws Exception .createReader(TopicName.get(topic)).get(); int segmentCount = 0; while (reader.hasMoreEvents()) { + @Cleanup("release") Message message = reader.readNextAsync() .get(5, TimeUnit.SECONDS); if (topic.equals(message.getValue().getTopicName())) { @@ -364,6 +365,7 @@ private void verifySnapshotSegmentsIndexSize(String topic, int size) throws Exce .createReader(TopicName.get(topic)).get(); int indexCount = 0; while (reader.hasMoreEvents()) { + @Cleanup("release") Message message = reader.readNextAsync() .get(5, TimeUnit.SECONDS); if (topic.equals(message.getValue().getTopicName())) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 14cc813a17ddd..ccf99936439dd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -715,7 +715,9 @@ public void testTransactionBufferIndexSystemTopic() throws Exception { indexesWriter.write(SNAPSHOT_INDEX, transactionBufferTransactionBufferSnapshotIndexes); assertTrue(indexesReader.hasMoreEvents()); - transactionBufferTransactionBufferSnapshotIndexes = indexesReader.readNext().getValue(); + @Cleanup("release") + Message message = indexesReader.readNext(); + transactionBufferTransactionBufferSnapshotIndexes = message.getValue(); assertEquals(transactionBufferTransactionBufferSnapshotIndexes.getTopicName(), SNAPSHOT_INDEX); assertEquals(transactionBufferTransactionBufferSnapshotIndexes.getIndexList().size(), 5); assertNull(transactionBufferTransactionBufferSnapshotIndexes.getSnapshot()); From 1cdb5db8400025590fc87124241f4fafcf6e5981 Mon Sep 17 00:00:00 2001 From: VisionXu Date: Fri, 3 Jan 2025 16:42:02 +0800 Subject: [PATCH 236/327] [fix][test]: Flaky-test: GetPartitionMetadataMultiBrokerTest.testCompatibilityDifferentBrokersForNonPersistentTopic (#23666) --- .../broker/admin/GetPartitionMetadataMultiBrokerTest.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java index d1eeabdb3d7cc..1b4732f69d4d2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java @@ -253,6 +253,10 @@ public void testCompatibilityDifferentBrokersForNonPersistentTopic(boolean confi boolean isUsingHttpLookup) throws Exception { modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); + // Verify: the method "getPartitionsForTopic(topic, false, true)" will fallback + // to "getPartitionsForTopic(topic, true)" behavior. + int lookupPermitsBefore = getLookupRequestPermits(); + // Initialize the connections of internal Pulsar Client. PulsarClientImpl client1 = (PulsarClientImpl) pulsar1.getClient(); PulsarClientImpl client2 = (PulsarClientImpl) pulsar2.getClient(); @@ -270,9 +274,6 @@ public void testCompatibilityDifferentBrokersForNonPersistentTopic(boolean confi field.set(clientCnx, false); } } - // Verify: the method "getPartitionsForTopic(topic, false, true)" will fallback - // to "getPartitionsForTopic(topic, true)" behavior. - int lookupPermitsBefore = getLookupRequestPermits(); // Verify: we will not get an un-support error. PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); From f199e8805f517373dbcc0f0c4a132218ecc24f0a Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 3 Jan 2025 17:56:33 +0800 Subject: [PATCH 237/327] [improve][client] PIP-393: Support configuring NegativeAckPrecisionBitCnt while building consumer. (#23804) --- .../pulsar/client/impl/NegativeAcksTest.java | 47 +++++++++++++++++++ .../pulsar/client/api/ConsumerBuilder.java | 13 +++++ .../client/impl/ConsumerBuilderImpl.java | 7 +++ 3 files changed, 67 insertions(+) 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 f8bc30f09667c..7ab3e545e981e 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 @@ -545,4 +545,51 @@ public void testMultiTopicConsumerConcurrentRedeliverAndReceive() throws Excepti consumer.close(); admin.topics().deletePartitionedTopic("persistent://public/default/" + topic); } + + @DataProvider(name = "negativeAckPrecisionBitCnt") + public Object[][] negativeAckPrecisionBitCnt() { + return new Object[][]{ + {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}, {11}, {12} + }; + } + + /** + * When negativeAckPrecisionBitCnt is greater than 0, the lower bits of the redelivery time will be truncated + * to reduce the memory occupation. If set to k, the redelivery time will be bucketed by 2^k ms, resulting in + * the redelivery time could be earlier(no later) than the expected time no more than 2^k ms. + * @throws Exception if an error occurs + */ + @Test(dataProvider = "negativeAckPrecisionBitCnt") + public void testConfigureNegativeAckPrecisionBitCnt(int negativeAckPrecisionBitCnt) throws Exception { + String topic = BrokerTestUtil.newUniqueName("testConfigureNegativeAckPrecisionBitCnt"); + long timeDeviation = 1L << negativeAckPrecisionBitCnt; + long delayInMs = 2000; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub1") + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .negativeAckRedeliveryDelay(delayInMs, TimeUnit.MILLISECONDS) + .negativeAckRedeliveryDelayPrecision(negativeAckPrecisionBitCnt) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + producer.sendAsync("test-0"); + producer.flush(); + + // receive the message and negative ack + consumer.negativeAcknowledge(consumer.receive()); + long expectedTime = System.currentTimeMillis() + delayInMs; + + // receive the redelivered message and calculate the time deviation + // assert that the redelivery time is no earlier than the `expected time - timeDeviation` + Message msg1 = consumer.receive(); + assertTrue(System.currentTimeMillis() >= expectedTime - timeDeviation); + assertNotNull(msg1); + } } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index 3ce12b7741a8f..ed77652c82340 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -243,6 +243,19 @@ public interface ConsumerBuilder extends Cloneable { */ ConsumerBuilder negativeAckRedeliveryDelay(long redeliveryDelay, TimeUnit timeUnit); + /** + * Sets the redelivery time precision bit count. The lower bits of the redelivery time will be + * trimmed to reduce the memory occupation. The default value is 8, which means the redelivery time + * will be bucketed by 256ms, the redelivery time could be earlier(no later) than the expected time, + * but no more than 256ms. If set to k, the redelivery time will be bucketed by 2^k ms. + * If the value is 0, the redelivery time will be accurate to ms. + * + * @param negativeAckPrecisionBitCnt + * The redelivery time precision bit count. + * @return the consumer builder instance + */ + ConsumerBuilder negativeAckRedeliveryDelayPrecision(int negativeAckPrecisionBitCount); + /** * Select the subscription type to be used when subscribing to a topic. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java index 35f772028f17a..478f93b56a0d2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java @@ -281,6 +281,13 @@ public ConsumerBuilder negativeAckRedeliveryDelay(long redeliveryDelay, TimeU return this; } + @Override + public ConsumerBuilder negativeAckRedeliveryDelayPrecision(int negativeAckPrecisionBitCount) { + checkArgument(negativeAckPrecisionBitCount >= 0, "negativeAckPrecisionBitCount needs to be >= 0"); + conf.setNegativeAckPrecisionBitCnt(negativeAckPrecisionBitCount); + return this; + } + @Override public ConsumerBuilder subscriptionType(@NonNull SubscriptionType subscriptionType) { conf.setSubscriptionType(subscriptionType); From 1f7a79f4c240c8075c217ac417dffa5bc72a4d5f Mon Sep 17 00:00:00 2001 From: danpi Date: Sat, 4 Jan 2025 02:04:24 +0800 Subject: [PATCH 238/327] [fix][admin] Fix exception thrown in getMessageId method (#23784) Co-authored-by: houbonan --- .../apache/pulsar/broker/admin/impl/PersistentTopicsBase.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 4b9fea05cd885..eed667d499000 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -2776,6 +2776,7 @@ public void readEntryFailed(ManagedLedgerException exception, if (exception instanceof ManagedLedgerException.LedgerNotExistException) { results.completeExceptionally( new RestException(Status.NOT_FOUND, "Message id not found")); + return; } results.completeExceptionally(new RestException(exception)); } From e0eaca991f28a3c41eef6a3178d15fcb422658b0 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 3 Jan 2025 13:50:07 -0800 Subject: [PATCH 239/327] [improve][ci] Move ZkSessionExpireTest to flaky group to unblock CI (#23810) --- .../org/apache/pulsar/broker/service/ZkSessionExpireTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java index 143557b008b23..dd0d7c423b643 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java @@ -39,7 +39,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "broker") +@Test(groups = "flaky") public class ZkSessionExpireTest extends NetworkErrorTestBase { private java.util.function.Consumer settings; From 2bd97845bdbc11733ee5ff2eaa5cd16e96129180 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sun, 5 Jan 2025 10:51:08 -0800 Subject: [PATCH 240/327] [improve] Upgrade to Netty 4.1.116.Final and io_uring to 0.0.26.Final (#23813) --- buildtools/pom.xml | 2 +- .../server/src/assemble/LICENSE.bin.txt | 46 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 44 +++++++++--------- pom.xml | 4 +- 4 files changed, 48 insertions(+), 48 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 9fb953929c3c9..100ed1d973783 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -47,7 +47,7 @@ 4.1 10.14.2 3.1.2 - 4.1.115.Final + 4.1.116.Final 4.2.3 32.1.2-jre 1.10.12 diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 2182142cd6428..92b1e88f85280 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -293,26 +293,26 @@ The Apache Software License, Version 2.0 - org.apache.commons-commons-lang3-3.11.jar - org.apache.commons-commons-text-1.10.0.jar * Netty - - io.netty-netty-buffer-4.1.115.Final.jar - - io.netty-netty-codec-4.1.115.Final.jar - - io.netty-netty-codec-dns-4.1.115.Final.jar - - io.netty-netty-codec-http-4.1.115.Final.jar - - io.netty-netty-codec-http2-4.1.115.Final.jar - - io.netty-netty-codec-socks-4.1.115.Final.jar - - io.netty-netty-codec-haproxy-4.1.115.Final.jar - - io.netty-netty-common-4.1.115.Final.jar - - io.netty-netty-handler-4.1.115.Final.jar - - io.netty-netty-handler-proxy-4.1.115.Final.jar - - io.netty-netty-resolver-4.1.115.Final.jar - - io.netty-netty-resolver-dns-4.1.115.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.115.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.115.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.115.Final.jar - - io.netty-netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar - - io.netty-netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar - - io.netty-netty-transport-native-unix-common-4.1.115.Final.jar + - io.netty-netty-buffer-4.1.116.Final.jar + - io.netty-netty-codec-4.1.116.Final.jar + - io.netty-netty-codec-dns-4.1.116.Final.jar + - io.netty-netty-codec-http-4.1.116.Final.jar + - io.netty-netty-codec-http2-4.1.116.Final.jar + - io.netty-netty-codec-socks-4.1.116.Final.jar + - io.netty-netty-codec-haproxy-4.1.116.Final.jar + - io.netty-netty-common-4.1.116.Final.jar + - io.netty-netty-handler-4.1.116.Final.jar + - io.netty-netty-handler-proxy-4.1.116.Final.jar + - io.netty-netty-resolver-4.1.116.Final.jar + - io.netty-netty-resolver-dns-4.1.116.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.116.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.116.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.116.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.116.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.116.Final.jar + - io.netty-netty-transport-native-epoll-4.1.116.Final-linux-aarch_64.jar + - io.netty-netty-transport-native-epoll-4.1.116.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.116.Final.jar - io.netty-netty-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 @@ -320,9 +320,9 @@ The Apache Software License, Version 2.0 - 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.incubator-netty-incubator-transport-classes-io_uring-0.0.24.Final.jar - - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar - - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.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 * Prometheus client - io.prometheus.jmx-collector-0.16.1.jar - io.prometheus-simpleclient-0.16.0.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 3333c9fe6ab66..989a98470593d 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -347,22 +347,22 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.26.0.jar * Netty - - netty-buffer-4.1.115.Final.jar - - netty-codec-4.1.115.Final.jar - - netty-codec-dns-4.1.115.Final.jar - - netty-codec-http-4.1.115.Final.jar - - netty-codec-socks-4.1.115.Final.jar - - netty-codec-haproxy-4.1.115.Final.jar - - netty-common-4.1.115.Final.jar - - netty-handler-4.1.115.Final.jar - - netty-handler-proxy-4.1.115.Final.jar - - netty-resolver-4.1.115.Final.jar - - netty-resolver-dns-4.1.115.Final.jar - - netty-transport-4.1.115.Final.jar - - netty-transport-classes-epoll-4.1.115.Final.jar - - netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar - - netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.115.Final.jar + - netty-buffer-4.1.116.Final.jar + - netty-codec-4.1.116.Final.jar + - netty-codec-dns-4.1.116.Final.jar + - netty-codec-http-4.1.116.Final.jar + - netty-codec-socks-4.1.116.Final.jar + - netty-codec-haproxy-4.1.116.Final.jar + - netty-common-4.1.116.Final.jar + - netty-handler-4.1.116.Final.jar + - netty-handler-proxy-4.1.116.Final.jar + - netty-resolver-4.1.116.Final.jar + - netty-resolver-dns-4.1.116.Final.jar + - netty-transport-4.1.116.Final.jar + - netty-transport-classes-epoll-4.1.116.Final.jar + - netty-transport-native-epoll-4.1.116.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.116.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.116.Final.jar - netty-tcnative-boringssl-static-2.0.69.Final.jar - netty-tcnative-boringssl-static-2.0.69.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.69.Final-linux-x86_64.jar @@ -370,12 +370,12 @@ The Apache Software License, Version 2.0 - netty-tcnative-boringssl-static-2.0.69.Final-osx-x86_64.jar - netty-tcnative-boringssl-static-2.0.69.Final-windows-x86_64.jar - netty-tcnative-classes-2.0.69.Final.jar - - netty-incubator-transport-classes-io_uring-0.0.24.Final.jar - - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar - - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.115.Final.jar - - netty-resolver-dns-native-macos-4.1.115.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.115.Final-osx-x86_64.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.116.Final.jar + - netty-resolver-dns-native-macos-4.1.116.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.116.Final-osx-x86_64.jar * Prometheus client - simpleclient-0.16.0.jar - simpleclient_log4j2-0.16.0.jar diff --git a/pom.xml b/pom.xml index 3cd9bd4b8d44d..047e014838f78 100644 --- a/pom.xml +++ b/pom.xml @@ -146,8 +146,8 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.7.1 - 4.1.115.Final - 0.0.24.Final + 4.1.116.Final + 0.0.26.Final 9.4.56.v20240826 2.5.2 2.42 From 420f62eef20aaef49ae404a683f2d5466ccfdec3 Mon Sep 17 00:00:00 2001 From: ran Date: Wed, 8 Jan 2025 00:34:45 +0800 Subject: [PATCH 241/327] [fix][broker] Remove failed OpAddEntry from pendingAddEntries (#23817) --- .../bookkeeper/mledger/impl/OpAddEntry.java | 1 + .../MangedLedgerInterceptorImpl2Test.java | 8 ++-- ... => ManagedLedgerInterceptorImplTest.java} | 40 ++++++++++++++----- 3 files changed, 35 insertions(+), 14 deletions(-) rename pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/{MangedLedgerInterceptorImplTest.java => ManagedLedgerInterceptorImplTest.java} (93%) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java index 2c00fd2cdf715..e0d35ce4e91bb 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java @@ -143,6 +143,7 @@ public void initiate() { payloadProcessorHandle = ml.getManagedLedgerInterceptor() .processPayloadBeforeLedgerWrite(this.getCtx(), duplicateBuffer); } catch (Exception e) { + ml.pendingAddEntries.remove(this); ReferenceCountUtil.safeRelease(duplicateBuffer); log.error("[{}] Error processing payload before ledger write", ml.getName(), e); this.failed(new ManagedLedgerException.ManagedLedgerInterceptException(e)); diff --git a/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/MangedLedgerInterceptorImpl2Test.java b/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/MangedLedgerInterceptorImpl2Test.java index 1be66a7f9d8f5..ec05f4dc381d7 100644 --- a/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/MangedLedgerInterceptorImpl2Test.java +++ b/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/MangedLedgerInterceptorImpl2Test.java @@ -19,7 +19,7 @@ package org.apache.bookkeeper.mledger.impl; import static org.testng.Assert.assertEquals; -import static org.apache.pulsar.broker.intercept.MangedLedgerInterceptorImplTest.TestPayloadProcessor; +import static org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImplTest.TestPayloadProcessor; import java.util.HashSet; import java.util.Set; import lombok.extern.slf4j.Slf4j; @@ -28,13 +28,13 @@ import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImpl; -import org.apache.pulsar.broker.intercept.MangedLedgerInterceptorImplTest; +import org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImplTest; import org.apache.pulsar.common.intercept.ManagedLedgerPayloadProcessor; import org.awaitility.Awaitility; import org.testng.annotations.Test; /*** - * Differ to {@link MangedLedgerInterceptorImplTest}, this test can call {@link ManagedLedgerImpl}'s methods modified + * Differ to {@link ManagedLedgerInterceptorImplTest}, this test can call {@link ManagedLedgerImpl}'s methods modified * by "default". */ @Slf4j @@ -73,7 +73,7 @@ public void testCurrentLedgerSizeCorrectIfHasInterceptor() throws Exception { switchLedgerManually(ledger); // verify. - assertEquals(currentLedgerSize, MangedLedgerInterceptorImplTest.calculatePreciseSize(ledger)); + assertEquals(currentLedgerSize, ManagedLedgerInterceptorImplTest.calculatePreciseSize(ledger)); // cleanup. cursor.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java similarity index 93% rename from pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java rename to pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java index 26b2d52c194ff..3866130c64be5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java @@ -21,7 +21,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.fail; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -33,6 +32,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import lombok.Cleanup; import org.apache.bookkeeper.mledger.AsyncCallbacks; @@ -59,8 +59,8 @@ import org.testng.annotations.Test; @Test(groups = "broker") -public class MangedLedgerInterceptorImplTest extends MockedBookKeeperTestCase { - private static final Logger log = LoggerFactory.getLogger(MangedLedgerInterceptorImplTest.class); +public class ManagedLedgerInterceptorImplTest extends MockedBookKeeperTestCase { + private static final Logger log = LoggerFactory.getLogger(ManagedLedgerInterceptorImplTest.class); public static class TestPayloadProcessor implements ManagedLedgerPayloadProcessor { @Override @@ -446,26 +446,33 @@ public Processor inputProcessor() { return new Processor() { @Override public ByteBuf process(Object contextObj, ByteBuf inputPayload) { - throw new RuntimeException(failureMsg); + Commands.skipBrokerEntryMetadataIfExist(inputPayload); + if (inputPayload.readBoolean()) { + throw new RuntimeException(failureMsg); + } + return inputPayload; } @Override public void release(ByteBuf processedPayload) { // no-op - fail("the release method can't be reached"); } }; } }))); var ledger = factory.open("testManagedLedgerPayloadProcessorFailure", config); - var countDownLatch = new CountDownLatch(1); + int count = 10; + var countDownLatch = new CountDownLatch(count); + var successCount = new AtomicInteger(0); var expectedException = new ArrayList(); - ledger.asyncAddEntry("test".getBytes(), 1, 1, new AsyncCallbacks.AddEntryCallback() { + + var addEntryCallback = new AsyncCallbacks.AddEntryCallback() { @Override public void addComplete(Position position, ByteBuf entryData, Object ctx) { entryData.release(); countDownLatch.countDown(); + successCount.incrementAndGet(); } @Override @@ -474,10 +481,23 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { expectedException.add(exception); countDownLatch.countDown(); } - }, null); + }; + + for (int i = 0; i < count; i++) { + if (i % 2 == 0) { + ledger.asyncAddEntry(Unpooled.buffer().writeBoolean(true), addEntryCallback, null); + } else { + ledger.asyncAddEntry(Unpooled.buffer().writeBoolean(false), addEntryCallback, null); + } + } + countDownLatch.await(); - assertEquals(expectedException.size(), 1); - assertEquals(expectedException.get(0).getCause().getMessage(), failureMsg); + assertEquals(expectedException.size(), count / 2); + assertEquals(successCount.get(), count / 2); + for (Exception e : expectedException) { + assertEquals(e.getCause().getMessage(), failureMsg); + } + ledger.close(); } } From 6d59b1a292eea1469c606a346b4a8e32afc33254 Mon Sep 17 00:00:00 2001 From: ran Date: Thu, 9 Jan 2025 04:05:21 +0800 Subject: [PATCH 242/327] [fix][test] Remove useless test code (#23823) --- .../broker/intercept/ManagedLedgerInterceptorImplTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java index 3866130c64be5..b57b5ce94be42 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java @@ -470,7 +470,6 @@ public void release(ByteBuf processedPayload) { var addEntryCallback = new AsyncCallbacks.AddEntryCallback() { @Override public void addComplete(Position position, ByteBuf entryData, Object ctx) { - entryData.release(); countDownLatch.countDown(); successCount.incrementAndGet(); } From 9149720841c15be2c83579e783abb959d34a23c1 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Thu, 9 Jan 2025 12:40:10 +0800 Subject: [PATCH 243/327] [improve][pip] PIP-401: Support set batching configurations for Pulsar Functions&Sources (#23793) --- pip/pip-401.md | 141 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 141 insertions(+) create mode 100644 pip/pip-401.md diff --git a/pip/pip-401.md b/pip/pip-401.md new file mode 100644 index 0000000000000..8f1bc7851f7da --- /dev/null +++ b/pip/pip-401.md @@ -0,0 +1,141 @@ +# PIP-401: Support set batching configurations for Pulsar Functions&Sources + +# Background knowledge + +Pulsar Functions and Sources enable the batching feature hard-coded, and also set the `batchingMaxPublishDelay` to 10ms, it only +supports set the `batch-builder` for now, this is not suitable for all the use cases, and also not feasible for users. + +# Motivation + +Support setting batching configurations for Pulsar Functions&Sources, to make it more flexible and suitable for users. + +# Goals + +## In Scope + +- Support setting batching configurations for Pulsar Functions&Sources. + +# High Level Design + +Make users able to enable&disable batching and set batching configurations for Pulsar Functions&Sources. + +# Detailed Design + +## Design & Implementation Details + +- Add a new message `BatchingSpec` with below fields in `Function.proto`, and add it as a new filed `batchingSpec` to the `ProducerSpec` message + - `bool enabled` + - `int32 batchingMaxPublishDelayMs` + - `int32 roundRobinRouterBatchingPartitionSwitchFrequency` + - `int32 batchingMaxMessages` + - `int32 batchingMaxBytes` + - `string batchBuilder` +- Add a new class `BatchingConfig` with below fields and add it as a new field `batchingConfig` to the `ProducerConfig`: + - `bool enabled` + - `int batchingMaxPublishDelayMs` + - `int roundRobinRouterBatchingPartitionSwitchFrequency` + - `int batchingMaxMessages` + - `int batchingMaxBytes` + - `String batchBuilder` + +And related logic also will be added: +- convert the `batchingSpec` field of the `ProducerSpec` from `FunctionDetails` to the `batchingConfig` field of the `ProducerConfig` and vice versa + +To keep the compatibility, when the `batchingSpec` of the `ProducerSpec` is null when creating the `ProducerConfig` from the `ProducerSpec`, +the `batchingConfig` field will be fallback to: `BatchingConfig(enabled=true, batchingMaxPublishDelayMs=10)`. + +After the changes, users can pass the batching configurations when creating the functions and sources, like below using CLI arguments: + +```shell +./bin/pulsar-admin functions create \ + --tenant public \ + --namespace default \ + --name test-java \ + --className org.apache.pulsar.functions.api.examples.ExclamationFunction \ + --inputs persistent://public/default/test-java-input \ + --producer-config '{"batchingConfig": {"enabled": true, "batchingMaxPublishDelayMs": 100, "roundRobinRouterBatchingPartitionSwitchFrequency": 10, "batchingMaxMessages": 1000}}' \ + --jar /pulsar/examples/api-examples.jar +``` + +```shell +./bin/pulsar-admin sources create \ + --name data-generator-source \ + --source-type data-generator \ + --destination-topic-name persistent://public/default/data-source-topic \ + --producer-config '{"batchingConfig": {"enabled": true, "batchingMaxPublishDelayMs": 100, "roundRobinRouterBatchingPartitionSwitchFrequency": 10, "batchingMaxMessages": 1000}}' \ + --source-config '{"sleepBetweenMessages": "1000"}' +``` + +Users can also use the function config file to set the batching configs for functions: + +```yaml +tenant: "public" +namespace: "default" +name: "test-java" +jar: "/pulsar/examples/api-examples.jar" +className: "org.apache.pulsar.functions.api.examples.ExclamationFunction" +inputs: ["persistent://public/default/test-java-input"] +output: "persistent://public/default/test-java-output" +autoAck: true +parallelism: 1 +producerConfig: + batchingConfig: + enabled: true + batchingMaxPublishDelayMs: 100 + roundRobinRouterBatchingPartitionSwitchFrequency: 10 + batchingMaxMessages: 1000 +``` + +And use source config file to set the batching configs for sources: + +```yaml +tenant: "public" +namespace: "default" +name: "data-generator-source" +topicName: "persistent://public/default/data-source-topic" +archive: "builtin://data-generator" +parallelism: 1 +configs: + sleepBetweenMessages: "5000" +producerConfig: + batchingConfig: + enabled: true + batchingMaxPublishDelayMs: 100 + roundRobinRouterBatchingPartitionSwitchFrequency: 10 + batchingMaxMessages: 1000 +``` + +## Public-facing Changes + +### CLI + + +# Monitoring + + +# Security Considerations + + +# Backward & Forward Compatibility + +## Revert + +No changes are needed to revert to the previous version. + +## Upgrade + +No other changes are needed to upgrade to the new version. + +# Alternatives + +None + +# General Notes + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/olx4xm8cdy43omp5c0jm44sj1gp0grcr +* Mailing List voting thread: https://lists.apache.org/thread/vhq6ox4nh2rx59yoxowftqzv8f9lnm4q From 2eb4eabc84f68fef5b29d894631c7c23d06ec3af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 9 Jan 2025 21:05:39 +0800 Subject: [PATCH 244/327] [improve][broker] Optimize subscription seek (cursor reset) by timestamp (#22792) Co-authored-by: Lari Hotari --- .../bookkeeper/mledger/ManagedCursor.java | 25 ++ .../mledger/impl/ManagedCursorImpl.java | 56 +++- .../mledger/impl/ManagedCursorTest.java | 291 ++++++++++++++++++ .../pulsar/broker/ServiceConfiguration.java | 18 ++ .../persistent/PersistentMessageFinder.java | 69 ++++- .../persistent/PersistentSubscription.java | 19 +- .../service/PersistentMessageFinderTest.java | 242 ++++++++++++++- 7 files changed, 690 insertions(+), 30 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index 042e03998696c..4e5e12365480c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -660,6 +660,31 @@ void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate condition, FindEntryCallback callback, Object ctx, boolean isFindFromLedger); + + /** + * Find the newest entry that matches the given predicate. + * + * @param constraint + * search only active entries or all entries + * @param condition + * predicate that reads an entry an applies a condition + * @param callback + * callback object returning the resultant position + * @param startPosition + * start position to search from. + * @param endPosition + * end position to search to. + * @param ctx + * opaque context + * @param isFindFromLedger + * find the newest entry from ledger + */ + default void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate condition, + Position startPosition, Position endPosition, FindEntryCallback callback, + Object ctx, boolean isFindFromLedger) { + asyncFindNewestMatching(constraint, condition, callback, ctx, isFindFromLedger); + } + /** * reset the cursor to specified position to enable replay of messages. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 934bfba4b0d81..50f5f36b2d53d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1272,27 +1272,55 @@ public void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate @Override public void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate condition, FindEntryCallback callback, Object ctx, boolean isFindFromLedger) { - OpFindNewest op; - Position startPosition = null; - long max = 0; + asyncFindNewestMatching(constraint, condition, null, null, callback, ctx, + isFindFromLedger); + } + + + @Override + public void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate condition, + Position start, Position end, FindEntryCallback callback, + Object ctx, boolean isFindFromLedger) { + Position startPosition; switch (constraint) { - case SearchAllAvailableEntries: - startPosition = getFirstPosition(); - max = ledger.getNumberOfEntries() - 1; - break; - case SearchActiveEntries: - startPosition = ledger.getNextValidPosition(markDeletePosition); - max = getNumberOfEntriesInStorage(); - break; - default: - callback.findEntryFailed(new ManagedLedgerException("Unknown position constraint"), Optional.empty(), ctx); - return; + case SearchAllAvailableEntries -> + startPosition = start == null ? getFirstPosition() : start; + case SearchActiveEntries -> { + if (start == null) { + startPosition = ledger.getNextValidPosition(markDeletePosition); + } else { + startPosition = start; + startPosition = startPosition.compareTo(markDeletePosition) <= 0 + ? ledger.getNextValidPosition(startPosition) : startPosition; + } + } + default -> { + callback.findEntryFailed( + new ManagedLedgerException("Unknown position constraint"), Optional.empty(), ctx); + return; + } } + // startPosition can't be null, should never go here. if (startPosition == null) { callback.findEntryFailed(new ManagedLedgerException("Couldn't find start position"), Optional.empty(), ctx); return; } + // Calculate the end position + Position endPosition = end == null ? ledger.lastConfirmedEntry : end; + endPosition = endPosition.compareTo(ledger.lastConfirmedEntry) > 0 ? ledger.lastConfirmedEntry : endPosition; + // Calculate the number of entries between the startPosition and endPosition + long max = 0; + if (startPosition.compareTo(endPosition) <= 0) { + max = ledger.getNumberOfEntries(Range.closed(startPosition, endPosition)); + } + + if (max <= 0) { + callback.findEntryComplete(null, ctx); + return; + } + + OpFindNewest op; if (isFindFromLedger) { op = new OpFindNewest(this.ledger, startPosition, condition, max, callback, ctx); } else { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 69b74fcf8f5c1..d3ea98131ad8f 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -4873,6 +4873,297 @@ public void operationFailed(ManagedLedgerException exception) { assertEquals(cursor.getReadPosition(), markDeletedPosition.getNext()); } + @Test + public void testFindNewestMatching_SearchAllAvailableEntries_ByStartAndEnd() throws Exception { + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setMaxEntriesPerLedger(2); + managedLedgerConfig.setMinimumRolloverTime(0, TimeUnit.MILLISECONDS); + @Cleanup + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testFindNewestMatching_SearchAllAvailableEntries_ByStartAndEnd", managedLedgerConfig); + @Cleanup + ManagedCursor managedCursor = ledger.openCursor("test"); + + Position position = ledger.addEntry("test".getBytes(Encoding)); + Position position1 = ledger.addEntry("test1".getBytes(Encoding)); + Position position2 = ledger.addEntry("test2".getBytes(Encoding)); + Position position3 = ledger.addEntry("test3".getBytes(Encoding)); + + Predicate condition = entry -> { + try { + Position p = entry.getPosition(); + return p.compareTo(position1) <= 0; + } finally { + entry.release(); + } + }; + + // find the newest entry with start and end position + AtomicBoolean failed = new AtomicBoolean(false); + CountDownLatch latch = new CountDownLatch(1); + AtomicReference positionRef = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchAllAvailableEntries, condition, position, position2, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef.set(position); + latch.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed.set(true); + latch.countDown(); + } + }, null, true); + + latch.await(); + assertFalse(failed.get()); + assertNotNull(positionRef.get()); + assertEquals(positionRef.get(), position1); + + // find the newest entry with start + AtomicBoolean failed1 = new AtomicBoolean(false); + CountDownLatch latch1 = new CountDownLatch(1); + AtomicReference positionRef1 = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchAllAvailableEntries, condition, position, null, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef1.set(position); + latch1.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed1.set(true); + latch1.countDown(); + } + }, null, true); + latch1.await(); + assertFalse(failed1.get()); + assertNotNull(positionRef1.get()); + assertEquals(positionRef1.get(), position1); + + // find the newest entry with end + AtomicBoolean failed2 = new AtomicBoolean(false); + CountDownLatch latch2 = new CountDownLatch(1); + AtomicReference positionRef2 = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchAllAvailableEntries, condition, null, position2, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef2.set(position); + latch2.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed2.set(true); + latch2.countDown(); + } + }, null, true); + latch2.await(); + assertFalse(failed2.get()); + assertNotNull(positionRef2.get()); + assertEquals(positionRef2.get(), position1); + + // find the newest entry without start and end position + AtomicBoolean failed3 = new AtomicBoolean(false); + CountDownLatch latch3 = new CountDownLatch(1); + AtomicReference positionRef3 = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchAllAvailableEntries, condition, null, null, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef3.set(position); + latch3.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed3.set(true); + latch3.countDown(); + } + }, null, true); + latch3.await(); + assertFalse(failed3.get()); + assertNotNull(positionRef3.get()); + assertEquals(positionRef3.get(), position1); + + // find position3 + AtomicBoolean failed4 = new AtomicBoolean(false); + CountDownLatch latch4 = new CountDownLatch(1); + AtomicReference positionRef4 = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchAllAvailableEntries, entry -> { + try { + Position p = entry.getPosition(); + return p.compareTo(position3) <= 0; + } finally { + entry.release(); + } + }, position3, position3, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef4.set(position); + latch4.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed4.set(true); + latch4.countDown(); + } + }, null, true); + latch4.await(); + assertFalse(failed4.get()); + assertNotNull(positionRef4.get()); + assertEquals(positionRef4.get(), position3); + } + + + @Test + public void testFindNewestMatching_SearchActiveEntries_ByStartAndEnd() throws Exception { + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setMaxEntriesPerLedger(2); + managedLedgerConfig.setMinimumRolloverTime(0, TimeUnit.MILLISECONDS); + @Cleanup + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testFindNewestMatching_SearchActiveEntries_ByStartAndEnd", managedLedgerConfig); + @Cleanup + ManagedCursorImpl managedCursor = (ManagedCursorImpl) ledger.openCursor("test"); + + Position position = ledger.addEntry("test".getBytes(Encoding)); + Position position1 = ledger.addEntry("test1".getBytes(Encoding)); + Position position2 = ledger.addEntry("test2".getBytes(Encoding)); + Position position3 = ledger.addEntry("test3".getBytes(Encoding)); + Position position4 = ledger.addEntry("test4".getBytes(Encoding)); + managedCursor.markDelete(position1); + assertEquals(managedCursor.getNumberOfEntries(), 3); + + Predicate condition = entry -> { + try { + Position p = entry.getPosition(); + return p.compareTo(position3) <= 0; + } finally { + entry.release(); + } + }; + + // find the newest entry with start and end position + AtomicBoolean failed = new AtomicBoolean(false); + CountDownLatch latch = new CountDownLatch(1); + AtomicReference positionRef = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchActiveEntries, condition, position2, position4, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef.set(position); + latch.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed.set(true); + latch.countDown(); + } + }, null, true); + latch.await(); + assertFalse(failed.get()); + assertNotNull(positionRef.get()); + assertEquals(positionRef.get(), position3); + + // find the newest entry with start + AtomicBoolean failed1 = new AtomicBoolean(false); + CountDownLatch latch1 = new CountDownLatch(1); + AtomicReference positionRef1 = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchActiveEntries, condition, position2, null, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef1.set(position); + latch1.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed1.set(true); + latch1.countDown(); + } + }, null, true); + + latch1.await(); + assertFalse(failed1.get()); + assertNotNull(positionRef1.get()); + assertEquals(positionRef1.get(), position3); + + // find the newest entry with end + AtomicBoolean failed2 = new AtomicBoolean(false); + CountDownLatch latch2 = new CountDownLatch(1); + AtomicReference positionRef2 = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchActiveEntries, condition, null, position4, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef2.set(position); + latch2.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed2.set(true); + latch2.countDown(); + } + }, null, true); + + latch2.await(); + assertFalse(failed2.get()); + assertNotNull(positionRef2.get()); + assertEquals(positionRef2.get(), position3); + + // find the newest entry without start and end position + AtomicBoolean failed3 = new AtomicBoolean(false); + CountDownLatch latch3 = new CountDownLatch(1); + AtomicReference positionRef3 = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchActiveEntries, condition, null, null, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef3.set(position); + latch3.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed3.set(true); + latch3.countDown(); + } + }, null, true); + latch3.await(); + assertFalse(failed3.get()); + assertNotNull(positionRef3.get()); + assertEquals(positionRef3.get(), position3); + + // find position4 + AtomicBoolean failed4 = new AtomicBoolean(false); + CountDownLatch latch4 = new CountDownLatch(1); + AtomicReference positionRef4 = new AtomicReference<>(); + managedCursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchActiveEntries, entry -> { + try { + Position p = entry.getPosition(); + return p.compareTo(position4) <= 0; + } finally { + entry.release(); + } + }, position4, position4, new AsyncCallbacks.FindEntryCallback() { + @Override + public void findEntryComplete(Position position, Object ctx) { + positionRef4.set(position); + latch4.countDown(); + } + + @Override + public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, Object ctx) { + failed4.set(true); + latch4.countDown(); + } + }, null, true); + latch4.await(); + assertFalse(failed4.get()); + assertNotNull(positionRef4.get()); + assertEquals(positionRef4.get(), position4); + } + @Test void testForceCursorRecovery() throws Exception { TestPulsarMockBookKeeper bk = new TestPulsarMockBookKeeper(executor); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 0b6f0e9418cf9..d27661d0ee65e 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2238,6 +2238,24 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Max time before triggering a rollover on a cursor ledger" ) private int managedLedgerCursorRolloverTimeInSeconds = 14400; + + @FieldContext( + category = CATEGORY_STORAGE_ML, + dynamic = true, + doc = "When resetting a subscription by timestamp, the broker will use the" + + " ledger closing timestamp metadata to determine the range of ledgers" + + " to search for the message where the subscription position is reset to. " + + " Since by default, the search condition is based on the message publish time provided by the " + + " client at the publish time, there will be some clock skew between the ledger closing timestamp " + + " metadata and the publish time." + + " This configuration is used to set the max clock skew between the ledger closing" + + " timestamp and the message publish time for finding the range of ledgers to open for searching." + + " The default value is 60000 milliseconds (60 seconds). When set to -1, the broker will not" + + " use the ledger closing timestamp metadata to determine the range of ledgers to search for the" + + " message." + ) + private int managedLedgerCursorResetLedgerCloseTimestampMaxClockSkewMillis = 60000; + @FieldContext( category = CATEGORY_STORAGE_ML, doc = "Max number of `acknowledgment holes` that are going to be persistently stored.\n\n" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java index 08273155e4cfa..5a4631cf205f1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java @@ -25,6 +25,9 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.Codec; @@ -37,6 +40,7 @@ public class PersistentMessageFinder implements AsyncCallbacks.FindEntryCallback { private final ManagedCursor cursor; private final String subName; + private final int ledgerCloseTimestampMaxClockSkewMillis; private final String topicName; private long timestamp = 0; @@ -48,19 +52,23 @@ public class PersistentMessageFinder implements AsyncCallbacks.FindEntryCallback AtomicIntegerFieldUpdater .newUpdater(PersistentMessageFinder.class, "messageFindInProgress"); - public PersistentMessageFinder(String topicName, ManagedCursor cursor) { + public PersistentMessageFinder(String topicName, ManagedCursor cursor, int ledgerCloseTimestampMaxClockSkewMillis) { this.topicName = topicName; this.cursor = cursor; this.subName = Codec.decode(cursor.getName()); + this.ledgerCloseTimestampMaxClockSkewMillis = ledgerCloseTimestampMaxClockSkewMillis; } public void findMessages(final long timestamp, AsyncCallbacks.FindEntryCallback callback) { - this.timestamp = timestamp; if (messageFindInProgressUpdater.compareAndSet(this, FALSE, TRUE)) { + this.timestamp = timestamp; if (log.isDebugEnabled()) { log.debug("[{}] Starting message position find at timestamp {}", subName, timestamp); } - + Pair range = + getFindPositionRange(cursor.getManagedLedger().getLedgersInfo().values(), + cursor.getManagedLedger().getLastConfirmedEntry(), timestamp, + ledgerCloseTimestampMaxClockSkewMillis); cursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchAllAvailableEntries, entry -> { try { long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); @@ -71,7 +79,7 @@ public void findMessages(final long timestamp, AsyncCallbacks.FindEntryCallback entry.release(); } return false; - }, this, callback, true); + }, range.getLeft(), range.getRight(), this, callback, true); } else { if (log.isDebugEnabled()) { log.debug("[{}][{}] Ignore message position find scheduled task, last find is still running", topicName, @@ -83,6 +91,59 @@ public void findMessages(final long timestamp, AsyncCallbacks.FindEntryCallback } } + public static Pair getFindPositionRange(Iterable ledgerInfos, + Position lastConfirmedEntry, long targetTimestamp, + int ledgerCloseTimestampMaxClockSkewMillis) { + if (ledgerCloseTimestampMaxClockSkewMillis < 0) { + // this feature is disabled when the value is negative + return Pair.of(null, null); + } + + long targetTimestampMin = targetTimestamp - ledgerCloseTimestampMaxClockSkewMillis; + long targetTimestampMax = targetTimestamp + ledgerCloseTimestampMaxClockSkewMillis; + + Position start = null; + Position end = null; + + LedgerInfo secondToLastLedgerInfo = null; + LedgerInfo lastLedgerInfo = null; + for (LedgerInfo info : ledgerInfos) { + if (!info.hasTimestamp()) { + // unexpected case, don't set start and end + return Pair.of(null, null); + } + secondToLastLedgerInfo = lastLedgerInfo; + lastLedgerInfo = info; + long closeTimestamp = info.getTimestamp(); + // For an open ledger, closeTimestamp is 0 + if (closeTimestamp == 0) { + end = null; + break; + } + if (closeTimestamp <= targetTimestampMin) { + start = PositionFactory.create(info.getLedgerId(), 0); + } else if (closeTimestamp > targetTimestampMax) { + // If the close timestamp is greater than the timestamp + end = PositionFactory.create(info.getLedgerId(), info.getEntries() - 1); + break; + } + } + // If the second-to-last ledger's close timestamp is less than the target timestamp, then start from the + // first entry of the last ledger when there are confirmed entries in the ledger + if (lastLedgerInfo != null && secondToLastLedgerInfo != null + && secondToLastLedgerInfo.getTimestamp() > 0 + && secondToLastLedgerInfo.getTimestamp() < targetTimestampMin) { + Position firstPositionInLedger = PositionFactory.create(lastLedgerInfo.getLedgerId(), 0); + if (lastConfirmedEntry != null + && lastConfirmedEntry.compareTo(firstPositionInLedger) >= 0) { + start = firstPositionInLedger; + } else { + start = lastConfirmedEntry; + } + } + return Pair.of(start, end); + } + private static final Logger log = LoggerFactory.getLogger(PersistentMessageFinder.class); @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index b5a1a9db5deb1..a96a7e75506eb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -134,6 +134,7 @@ public class PersistentSubscription extends AbstractSubscription { private volatile CompletableFuture fenceFuture; private volatile CompletableFuture inProgressResetCursorFuture; private volatile Boolean replicatedControlled; + private final ServiceConfiguration config; static Map getBaseCursorProperties(Boolean isReplicated) { return isReplicated != null && isReplicated ? REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES : @@ -156,6 +157,7 @@ public PersistentSubscription(PersistentTopic topic, String subscriptionName, Ma public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, Boolean replicated, Map subscriptionProperties) { this.topic = topic; + this.config = topic.getBrokerService().getPulsar().getConfig(); this.cursor = cursor; this.topicName = topic.getName(); this.subName = subscriptionName; @@ -166,7 +168,7 @@ public PersistentSubscription(PersistentTopic topic, String subscriptionName, Ma } this.subscriptionProperties = MapUtils.isEmpty(subscriptionProperties) ? Collections.emptyMap() : Collections.unmodifiableMap(subscriptionProperties); - if (topic.getBrokerService().getPulsar().getConfig().isTransactionCoordinatorEnabled() + if (config.isTransactionCoordinatorEnabled() && !isEventSystemTopic(TopicName.get(topicName)) && !ExtensibleLoadManagerImpl.isInternalTopic(topicName)) { this.pendingAckHandle = new PendingAckHandleImpl(this); @@ -203,7 +205,6 @@ public boolean isReplicated() { public boolean setReplicated(boolean replicated) { replicatedControlled = replicated; - ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); if (!replicated || !config.isEnableReplicatedSubscriptions()) { this.replicatedSubscriptionSnapshotCache = null; @@ -261,7 +262,6 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { case Shared: if (dispatcher == null || dispatcher.getType() != SubType.Shared) { previousDispatcher = dispatcher; - ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); if (config.isSubscriptionSharedUseClassicPersistentImplementation()) { dispatcher = new PersistentDispatcherMultipleConsumersClassic(topic, cursor, this); } else { @@ -290,7 +290,6 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { || !((StickyKeyDispatcher) dispatcher) .hasSameKeySharedPolicy(ksm)) { previousDispatcher = dispatcher; - ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); if (config.isSubscriptionKeySharedUseClassicPersistentImplementation()) { dispatcher = new PersistentStickyKeyDispatcherMultipleConsumersClassic(topic, cursor, @@ -426,7 +425,7 @@ public void acknowledgeMessage(List positions, AckType ackType, Map { if ((cursor.isMessageDeleted(position))) { pendingAckHandle.clearIndividualPosition(position); @@ -602,10 +601,9 @@ public CompletableFuture analyzeBacklog(Optional final EntryFilterSupport entryFilterSupport = dispatcher != null ? (EntryFilterSupport) dispatcher : new EntryFilterSupport(this); // we put some hard limits on the scan, in order to prevent denial of services - ServiceConfiguration configuration = topic.getBrokerService().getPulsar().getConfiguration(); - long maxEntries = configuration.getSubscriptionBacklogScanMaxEntries(); - long timeOutMs = configuration.getSubscriptionBacklogScanMaxTimeMs(); - int batchSize = configuration.getDispatcherMaxReadBatchSize(); + long maxEntries = config.getSubscriptionBacklogScanMaxEntries(); + long timeOutMs = config.getSubscriptionBacklogScanMaxTimeMs(); + int batchSize = config.getDispatcherMaxReadBatchSize(); AtomicReference firstPosition = new AtomicReference<>(); AtomicReference lastPosition = new AtomicReference<>(); final Predicate condition = entry -> { @@ -780,7 +778,8 @@ public void skipEntriesFailed(ManagedLedgerException exception, Object ctx) { @Override public CompletableFuture resetCursor(long timestamp) { CompletableFuture future = new CompletableFuture<>(); - PersistentMessageFinder persistentMessageFinder = new PersistentMessageFinder(topicName, cursor); + PersistentMessageFinder persistentMessageFinder = new PersistentMessageFinder(topicName, cursor, + config.getManagedLedgerCursorResetLedgerCloseTimestampMaxClockSkewMillis()); if (log.isDebugEnabled()) { log.debug("[{}][{}] Resetting subscription to timestamp {}", topicName, subName, timestamp); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java index 176a799292ac3..6f2f1f3a1a2c0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java @@ -59,6 +59,7 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor; import org.apache.pulsar.broker.service.persistent.PersistentMessageFinder; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -138,7 +139,7 @@ void reset() { } CompletableFuture findMessage(final Result result, final ManagedCursor c1, final long timestamp) { - PersistentMessageFinder messageFinder = new PersistentMessageFinder("topicname", c1); + PersistentMessageFinder messageFinder = new PersistentMessageFinder("topicname", c1, 0); final CompletableFuture future = new CompletableFuture<>(); messageFinder.findMessages(timestamp, new AsyncCallbacks.FindEntryCallback() { @@ -217,7 +218,7 @@ void testPersistentMessageFinder() throws Exception { assertNotEquals(result.position, null); assertEquals(result.position, lastPosition); - PersistentMessageFinder messageFinder = new PersistentMessageFinder("topicname", c1); + PersistentMessageFinder messageFinder = new PersistentMessageFinder("topicname", c1, 0); final AtomicBoolean ex = new AtomicBoolean(false); messageFinder.findEntryFailed(new ManagedLedgerException("failed"), Optional.empty(), new AsyncCallbacks.FindEntryCallback() { @@ -589,4 +590,241 @@ public void test() { resetCursorData.setExcluded(true); System.out.println(Entity.entity(resetCursorData, MediaType.APPLICATION_JSON)); } + + @Test + public void testGetFindPositionRange_EmptyLedgerInfos() { + List ledgerInfos = new ArrayList<>(); + Position lastConfirmedEntry = null; + long targetTimestamp = 2000; + Pair range = + PersistentMessageFinder.getFindPositionRange(ledgerInfos, lastConfirmedEntry, targetTimestamp, 0); + + assertNotNull(range); + assertNull(range.getLeft()); + assertNull(range.getRight()); + } + + @Test + public void testGetFindPositionRange_AllTimestampsLessThanTarget() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(1500).build()); + Position lastConfirmedEntry = PositionFactory.create(2, 9); + + long targetTimestamp = 2000; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 0); + + assertNotNull(range); + assertNotNull(range.getLeft()); + assertNull(range.getRight()); + assertEquals(range.getLeft(), PositionFactory.create(2, 0)); + } + + @Test + public void testGetFindPositionRange_LastTimestampIsZero() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(1500).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(0).build()); + Position lastConfirmedEntry = PositionFactory.create(3, 5); + + long targetTimestamp = 2000; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 0); + + assertNotNull(range); + assertNotNull(range.getLeft()); + assertNull(range.getRight()); + assertEquals(range.getLeft(), PositionFactory.create(3, 0)); + } + + @Test + public void testGetFindPositionRange_LastTimestampIsZeroWithNoEntries() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(1500).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(0).build()); + Position lastConfirmedEntry = PositionFactory.create(2, 9); + + long targetTimestamp = 2000; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 0); + + assertNotNull(range); + assertNotNull(range.getLeft()); + assertNull(range.getRight()); + assertEquals(range.getLeft(), PositionFactory.create(2, 9)); + } + + @Test + public void testGetFindPositionRange_AllTimestampsGreaterThanTarget() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(3000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(4000).build()); + Position lastConfirmedEntry = PositionFactory.create(2, 9); + + long targetTimestamp = 2000; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 0); + + assertNotNull(range); + assertNull(range.getLeft()); + assertNotNull(range.getRight()); + assertEquals(range.getRight(), PositionFactory.create(1, 9)); + } + + @Test + public void testGetFindPositionRange_MixedTimestamps() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(3000).build()); + Position lastConfirmedEntry = PositionFactory.create(3, 9); + + long targetTimestamp = 2500; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 0); + + assertNotNull(range); + assertNotNull(range.getLeft()); + assertNotNull(range.getRight()); + assertEquals(range.getLeft(), PositionFactory.create(3, 0)); + assertEquals(range.getRight(), PositionFactory.create(3, 9)); + } + + @Test + public void testGetFindPositionRange_TimestampAtBoundary() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(3000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); + Position lastConfirmedEntry = PositionFactory.create(4, 9); + + long targetTimestamp = 3000; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 0); + + assertNotNull(range); + assertNotNull(range.getLeft()); + assertNotNull(range.getRight()); + assertEquals(range.getLeft(), PositionFactory.create(3, 0)); + // there might be entries in the next ledger with the same timestamp as the target timestamp, even though + // the close timestamp of ledger 3 is equals to the target timestamp + assertEquals(range.getRight(), PositionFactory.create(4, 9)); + } + + @Test + public void testGetFindPositionRange_ClockSkew() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(2010).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(5).setTimestamp(0).build()); + Position lastConfirmedEntry = PositionFactory.create(5, 5); + + long targetTimestamp = 2009; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 10); + + assertNotNull(range); + assertNotNull(range.getLeft()); + assertNotNull(range.getRight()); + assertEquals(range.getLeft(), PositionFactory.create(1, 0)); + assertEquals(range.getRight(), PositionFactory.create(4, 9)); + } + + @Test + public void testGetFindPositionRange_ClockSkewCase2() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(3000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(5).setTimestamp(0).build()); + Position lastConfirmedEntry = PositionFactory.create(5, 5); + + long targetTimestamp = 2995; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 10); + + assertNotNull(range); + assertNotNull(range.getLeft()); + assertNotNull(range.getRight()); + assertEquals(range.getLeft(), PositionFactory.create(2, 0)); + assertEquals(range.getRight(), PositionFactory.create(4, 9)); + } + + @Test + public void testGetFindPositionRange_ClockSkewCase3() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(3000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(5).setTimestamp(0).build()); + Position lastConfirmedEntry = PositionFactory.create(5, 5); + + long targetTimestamp = 3005; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 10); + + assertNotNull(range); + assertNotNull(range.getLeft()); + assertNotNull(range.getRight()); + assertEquals(range.getLeft(), PositionFactory.create(2, 0)); + assertEquals(range.getRight(), PositionFactory.create(4, 9)); + } + + @Test + public void testGetFindPositionRange_FeatureDisabledWithNegativeClockSkew() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(2010).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(5).setTimestamp(0).build()); + Position lastConfirmedEntry = PositionFactory.create(5, 5); + + long targetTimestamp = 2009; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, -1); + + assertNotNull(range); + assertNull(range.getLeft()); + assertNull(range.getRight()); + } + + @Test + public void testGetFindPositionRange_SingleLedger() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setTimestamp(0).build()); + Position lastConfirmedEntry = PositionFactory.create(1, 5); + + long targetTimestamp = 2500; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 0); + + assertNotNull(range); + assertNull(range.getLeft()); + assertNull(range.getRight()); + } + + @Test + public void testGetFindPositionRange_SingleClosedLedger() { + List ledgerInfos = new ArrayList<>(); + ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + Position lastConfirmedEntry = PositionFactory.create(1, 9); + + long targetTimestamp = 2500; + Pair range = PersistentMessageFinder.getFindPositionRange(ledgerInfos, + lastConfirmedEntry, targetTimestamp, 0); + + assertNotNull(range); + assertNotNull(range.getLeft()); + assertNull(range.getRight()); + assertEquals(range.getLeft(), PositionFactory.create(1, 0)); + } } From 04e89fe2d841246e655bf875ba52cda2c2de0e3d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 9 Jan 2025 11:10:49 -0800 Subject: [PATCH 245/327] [fix][client] Prevent retry topic and dead letter topic producer leaks when sending of message fails (#23824) --- .../client/api/DeadLetterTopicTest.java | 93 ++++++ .../pulsar/client/api/RetryTopicTest.java | 166 ++++++--- .../pulsar/client/impl/ConsumerImpl.java | 315 +++++++++++------- .../client/impl/MultiTopicsConsumerImpl.java | 9 +- .../src/main/resources/findbugsExclude.xml | 4 + 5 files changed, 423 insertions(+), 164 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java index e46fddeacc117..ab26949c04fc6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; @@ -40,9 +41,11 @@ import lombok.Cleanup; import lombok.Data; import org.apache.avro.reflect.Nullable; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.impl.ConsumerBuilderImpl; import org.apache.pulsar.client.util.RetryMessageUtil; +import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1167,4 +1170,94 @@ public void testDeadLetterPolicyDeserialize() throws Exception { consumerBuilder.loadConf(config); assertEquals(((ConsumerBuilderImpl)consumerBuilder).getConf().getDeadLetterPolicy(), policy); } + + @Data + static class Payload { + String number; + + public Payload() { + + } + + public Payload(String number) { + this.number = number; + } + } + + @Data + static class PayloadIncompatible { + long number; + + public PayloadIncompatible() { + + } + + public PayloadIncompatible(long number) { + this.number = number; + } + } + + // reproduce issue reported in https://github.com/apache/pulsar/issues/20635#issuecomment-1709616321 + @Test + public void testCloseDeadLetterTopicProducerOnExceptionToPreventProducerLeak() throws Exception { + String namespace = BrokerTestUtil.newUniqueName("my-property/my-ns"); + admin.namespaces().createNamespace(namespace); + // don't enforce schema validation + admin.namespaces().setSchemaValidationEnforced(namespace, false); + // set schema compatibility strategy to always compatible + admin.namespaces().setSchemaCompatibilityStrategy(namespace, SchemaCompatibilityStrategy.ALWAYS_COMPATIBLE); + + Schema schema = Schema.AVRO(Payload.class); + Schema schemaIncompatible = Schema.AVRO(PayloadIncompatible.class); + String topic = BrokerTestUtil.newUniqueName("persistent://" + namespace + + "/testCloseDeadLetterTopicProducerOnExceptionToPreventProducerLeak"); + String dlqTopic = topic + "-DLQ"; + + // create topics + admin.topics().createNonPartitionedTopic(topic); + admin.topics().createNonPartitionedTopic(dlqTopic); + + AtomicInteger nackCounter = new AtomicInteger(0); + Consumer payloadConsumer = null; + try { + payloadConsumer = pulsarClient.newConsumer(schema).topic(topic) + .subscriptionType(SubscriptionType.Shared).subscriptionName("sub") + .ackTimeout(1, TimeUnit.SECONDS) + .negativeAckRedeliveryDelay(1, TimeUnit.MILLISECONDS) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(3).deadLetterTopic(dlqTopic).build()) + .messageListener((c, msg) -> { + if (nackCounter.incrementAndGet() < 10) { + c.negativeAcknowledge(msg); + } + }).subscribe(); + + // send a message to the topic with the incompatible schema + PayloadIncompatible payloadIncompatible = new PayloadIncompatible(123); + try (Producer producer = pulsarClient.newProducer(schemaIncompatible).topic(topic) + .create()) { + producer.send(payloadIncompatible); + } + + Thread.sleep(2000L); + + assertThat(pulsar.getBrokerService().getTopicReference(dlqTopic).get().getProducers().size()) + .describedAs("producer count of dlq topic %s should be <= 1 so that it doesn't leak producers", + dlqTopic) + .isLessThanOrEqualTo(1); + + } finally { + if (payloadConsumer != null) { + try { + payloadConsumer.close(); + } catch (PulsarClientException e) { + // ignore + } + } + } + + assertThat(pulsar.getBrokerService().getTopicReference(dlqTopic).get().getProducers().size()) + .describedAs("producer count of dlq topic %s should be 0 here", + dlqTopic) + .isEqualTo(0); + } } 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 cd598585c8e87..91b97fa475817 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 @@ -18,12 +18,12 @@ */ package org.apache.pulsar.client.api; +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.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -import java.lang.reflect.Field; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -36,11 +36,10 @@ import lombok.Data; import org.apache.avro.AvroRuntimeException; import org.apache.avro.reflect.Nullable; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.client.api.schema.GenericRecord; -import org.apache.pulsar.client.impl.ConsumerImpl; -import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.client.util.RetryMessageUtil; -import org.reflections.ReflectionUtils; +import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; @@ -617,10 +616,12 @@ public void testRetryTopicByCustomTopicName() throws Exception { @Test(timeOut = 30000L) public void testRetryTopicException() throws Exception { - final String topic = "persistent://my-property/my-ns/retry-topic"; + String retryLetterTopic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/retry-topic"); + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/input-topic"); final int maxRedeliveryCount = 2; final int sendMessages = 1; // subscribe before publish + @Cleanup Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .topic(topic) .subscriptionName("my-subscription") @@ -629,7 +630,7 @@ public void testRetryTopicException() throws Exception { .receiverQueueSize(100) .deadLetterPolicy(DeadLetterPolicy.builder() .maxRedeliverCount(maxRedeliveryCount) - .retryLetterTopic("persistent://my-property/my-ns/my-subscription-custom-Retry") + .retryLetterTopic(retryLetterTopic) .build()) .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); @@ -642,30 +643,16 @@ public void testRetryTopicException() throws Exception { } producer.close(); - // mock a retry producer exception when reconsumelater is called - MultiTopicsConsumerImpl multiTopicsConsumer = (MultiTopicsConsumerImpl) consumer; - List> consumers = multiTopicsConsumer.getConsumers(); - for (ConsumerImpl c : consumers) { - Set deadLetterPolicyField = - ReflectionUtils.getAllFields(c.getClass(), ReflectionUtils.withName("deadLetterPolicy")); - - if (deadLetterPolicyField.size() != 0) { - Field field = deadLetterPolicyField.iterator().next(); - field.setAccessible(true); - DeadLetterPolicy deadLetterPolicy = (DeadLetterPolicy) field.get(c); - deadLetterPolicy.setRetryLetterTopic("#persistent://invlaid-topic#"); - } - } + admin.topics().terminateTopic(retryLetterTopic); + Message message = consumer.receive(); log.info("consumer received message : {} {}", message.getMessageId(), new String(message.getData())); try { consumer.reconsumeLater(message, 1, TimeUnit.SECONDS); - } catch (PulsarClientException.InvalidTopicNameException e) { - assertEquals(e.getClass(), PulsarClientException.InvalidTopicNameException.class); - } catch (Exception e) { - fail("exception should be PulsarClientException.InvalidTopicNameException"); + fail("exception should be PulsarClientException.TopicTerminatedException"); + } catch (PulsarClientException.TopicTerminatedException e) { + // ok } - consumer.close(); } @@ -718,10 +705,12 @@ public void testRetryProducerWillCloseByConsumer() throws Exception { @Test(timeOut = 30000L) public void testRetryTopicExceptionWithConcurrent() throws Exception { - final String topic = "persistent://my-property/my-ns/retry-topic"; + String retryLetterTopic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/retry-topic"); + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/input-topic"); final int maxRedeliveryCount = 2; final int sendMessages = 10; // subscribe before publish + @Cleanup Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .topic(topic) .subscriptionName("my-subscription") @@ -730,7 +719,7 @@ public void testRetryTopicExceptionWithConcurrent() throws Exception { .receiverQueueSize(100) .deadLetterPolicy(DeadLetterPolicy.builder() .maxRedeliverCount(maxRedeliveryCount) - .retryLetterTopic("persistent://my-property/my-ns/my-subscription-custom-Retry") + .retryLetterTopic(retryLetterTopic) .build()) .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); @@ -739,24 +728,11 @@ public void testRetryTopicExceptionWithConcurrent() throws Exception { .topic(topic) .create(); for (int i = 0; i < sendMessages; i++) { - producer.newMessage().key("1").value(String.format("Hello Pulsar [%d]", i).getBytes()).send(); + producer.send(String.format("Hello Pulsar [%d]", i).getBytes()); } producer.close(); - // mock a retry producer exception when reconsumelater is called - MultiTopicsConsumerImpl multiTopicsConsumer = (MultiTopicsConsumerImpl) consumer; - List> consumers = multiTopicsConsumer.getConsumers(); - for (ConsumerImpl c : consumers) { - Set deadLetterPolicyField = - ReflectionUtils.getAllFields(c.getClass(), ReflectionUtils.withName("deadLetterPolicy")); - - if (deadLetterPolicyField.size() != 0) { - Field field = deadLetterPolicyField.iterator().next(); - field.setAccessible(true); - DeadLetterPolicy deadLetterPolicy = (DeadLetterPolicy) field.get(c); - deadLetterPolicy.setRetryLetterTopic("#persistent://invalid-topic#"); - } - } + admin.topics().terminateTopic(retryLetterTopic); List> messages = Lists.newArrayList(); for (int i = 0; i < sendMessages; i++) { @@ -769,16 +745,114 @@ public void testRetryTopicExceptionWithConcurrent() throws Exception { new Thread(() -> { try { consumer.reconsumeLater(message, 1, TimeUnit.SECONDS); - } catch (Exception ignore) { - - } finally { + } catch (PulsarClientException.TopicTerminatedException e) { + // ok latch.countDown(); + } catch (PulsarClientException e) { + // unexpected exception + fail("unexpected exception", e); } }).start(); } - latch.await(); + latch.await(sendMessages, TimeUnit.SECONDS); consumer.close(); } + @Data + static class Payload { + String number; + + public Payload() { + + } + + public Payload(String number) { + this.number = number; + } + } + + @Data + static class PayloadIncompatible { + long number; + + public PayloadIncompatible() { + + } + + public PayloadIncompatible(long number) { + this.number = number; + } + } + + // reproduce similar issue as reported in https://github.com/apache/pulsar/issues/20635#issuecomment-1709616321 + // but for retry topic + @Test + public void testCloseRetryLetterTopicProducerOnExceptionToPreventProducerLeak() throws Exception { + String namespace = BrokerTestUtil.newUniqueName("my-property/my-ns"); + admin.namespaces().createNamespace(namespace); + // don't enforce schema validation + admin.namespaces().setSchemaValidationEnforced(namespace, false); + // set schema compatibility strategy to always compatible + admin.namespaces().setSchemaCompatibilityStrategy(namespace, SchemaCompatibilityStrategy.ALWAYS_COMPATIBLE); + + Schema schema = Schema.AVRO(Payload.class); + Schema schemaIncompatible = Schema.AVRO( + PayloadIncompatible.class); + String topic = BrokerTestUtil.newUniqueName("persistent://" + namespace + + "/testCloseDeadLetterTopicProducerOnExceptionToPreventProducerLeak"); + String dlqTopic = topic + "-DLQ"; + String retryTopic = topic + "-RETRY"; + + // create topics + admin.topics().createNonPartitionedTopic(topic); + admin.topics().createNonPartitionedTopic(dlqTopic); + admin.topics().createNonPartitionedTopic(retryTopic); + + Consumer payloadConsumer = null; + try { + payloadConsumer = pulsarClient.newConsumer(schema).topic(topic) + .subscriptionType(SubscriptionType.Shared).subscriptionName("sub") + .ackTimeout(1, TimeUnit.SECONDS) + .negativeAckRedeliveryDelay(1, TimeUnit.MILLISECONDS) + .enableRetry(true) + .deadLetterPolicy(DeadLetterPolicy.builder().retryLetterTopic(retryTopic).maxRedeliverCount(3) + .deadLetterTopic(dlqTopic).build()) + .messageListener((c, msg) -> { + try { + c.reconsumeLater(msg, 1, TimeUnit.MILLISECONDS); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + }).subscribe(); + + // send a message to the topic with the incompatible schema + PayloadIncompatible payloadIncompatible = new PayloadIncompatible(123); + try (Producer producer = pulsarClient.newProducer(schemaIncompatible).topic(topic) + .create()) { + producer.send(payloadIncompatible); + } + + Thread.sleep(2000L); + + assertThat(pulsar.getBrokerService().getTopicReference(retryTopic).get().getProducers().size()) + .describedAs("producer count of retry topic %s should be <= 1 so that it doesn't leak producers", + retryTopic) + .isLessThanOrEqualTo(1); + + } finally { + if (payloadConsumer != null) { + try { + payloadConsumer.close(); + } catch (PulsarClientException e) { + // ignore + } + } + } + + assertThat(pulsar.getBrokerService().getTopicReference(retryTopic).get().getProducers().size()) + .describedAs("producer count of retry topic %s should be 0 here", + retryTopic) + .isEqualTo(0); + } } 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 86af4bdaf58c8..77a91a944ee6c 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 @@ -67,6 +67,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.Getter; @@ -202,8 +203,9 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final DeadLetterPolicy deadLetterPolicy; private volatile CompletableFuture> deadLetterProducer; - + private volatile int deadLetterProducerFailureCount; private volatile CompletableFuture> retryLetterProducer; + private volatile int retryLetterProducerFailureCount; private final ReadWriteLock createProducerLock = new ReentrantReadWriteLock(); protected volatile boolean paused; @@ -682,9 +684,8 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a return FutureUtil.failedFuture(exception); } - initRetryLetterProducerIfNeeded(); CompletableFuture result = new CompletableFuture<>(); - if (retryLetterProducer != null) { + if (initRetryLetterProducerIfNeeded() != null) { try { MessageImpl retryMessage = (MessageImpl) getMessageImpl(message); String originMessageIdStr = message.getMessageId().toString(); @@ -707,52 +708,61 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a MessageId finalMessageId = messageId; if (reconsumeTimes > this.deadLetterPolicy.getMaxRedeliverCount() && StringUtils.isNotBlank(deadLetterPolicy.getDeadLetterTopic())) { - initDeadLetterProducerIfNeeded(); - deadLetterProducer.thenAcceptAsync(dlqProducer -> { - TypedMessageBuilder typedMessageBuilderNew = - dlqProducer.newMessage(Schema.AUTO_PRODUCE_BYTES(retryMessage.getReaderSchema().get())) - .value(retryMessage.getData()) - .properties(propertiesMap); - copyMessageKeysIfNeeded(message, typedMessageBuilderNew); - typedMessageBuilderNew.sendAsync().thenAccept(msgId -> { - consumerDlqMessagesCounter.increment(); - - doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null).thenAccept(v -> { - result.complete(null); + initDeadLetterProducerIfNeeded().thenAcceptAsync(dlqProducer -> { + try { + TypedMessageBuilder typedMessageBuilderNew = + dlqProducer.newMessage( + Schema.AUTO_PRODUCE_BYTES(retryMessage.getReaderSchema().get())) + .value(retryMessage.getData()) + .properties(propertiesMap); + copyMessageKeysIfNeeded(message, typedMessageBuilderNew); + typedMessageBuilderNew.sendAsync().thenAccept(msgId -> { + consumerDlqMessagesCounter.increment(); + + doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null).thenAccept(v -> { + result.complete(null); + }).exceptionally(ex -> { + result.completeExceptionally(ex); + return null; + }); }).exceptionally(ex -> { result.completeExceptionally(ex); return null; }); - }).exceptionally(ex -> { - result.completeExceptionally(ex); - return null; - }); + } catch (Exception e) { + result.completeExceptionally(e); + } }, internalPinnedExecutor).exceptionally(ex -> { result.completeExceptionally(ex); - deadLetterProducer = null; return null; }); } else { assert retryMessage != null; - retryLetterProducer.thenAcceptAsync(rtlProducer -> { - TypedMessageBuilder typedMessageBuilderNew = rtlProducer - .newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) - .value(retryMessage.getData()) - .properties(propertiesMap); - if (delayTime > 0) { - typedMessageBuilderNew.deliverAfter(delayTime, unit); + initRetryLetterProducerIfNeeded().thenAcceptAsync(rtlProducer -> { + try { + TypedMessageBuilder typedMessageBuilderNew = rtlProducer + .newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) + .value(retryMessage.getData()) + .properties(propertiesMap); + if (delayTime > 0) { + typedMessageBuilderNew.deliverAfter(delayTime, unit); + } + copyMessageKeysIfNeeded(message, typedMessageBuilderNew); + typedMessageBuilderNew.sendAsync() + .thenCompose( + __ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null)) + .thenAccept(v -> { + result.complete(null); + }) + .exceptionally(ex -> { + result.completeExceptionally(ex); + return null; + }); + } catch (Exception e) { + result.completeExceptionally(e); } - copyMessageKeysIfNeeded(message, typedMessageBuilderNew); - typedMessageBuilderNew.sendAsync() - .thenCompose(__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null)) - .thenAccept(v -> result.complete(null)) - .exceptionally(ex -> { - result.completeExceptionally(ex); - return null; - }); }, internalPinnedExecutor).exceptionally(ex -> { result.completeExceptionally(ex); - retryLetterProducer = null; return null; }); } @@ -1099,10 +1109,29 @@ public void connectionFailed(PulsarClientException exception) { public synchronized CompletableFuture closeAsync() { CompletableFuture closeFuture = new CompletableFuture<>(); + ArrayList> closeFutures = new ArrayList<>(4); + closeFutures.add(closeFuture); + if (retryLetterProducer != null) { + closeFutures.add(retryLetterProducer.thenCompose(p -> p.closeAsync()).whenComplete((ignore, ex) -> { + if (ex != null) { + log.warn("Exception ignored in closing retryLetterProducer of consumer", ex); + } + })); + } + if (deadLetterProducer != null) { + closeFutures.add(deadLetterProducer.thenCompose(p -> p.closeAsync()).whenComplete((ignore, ex) -> { + if (ex != null) { + log.warn("Exception ignored in closing deadLetterProducer of consumer", ex); + } + })); + } + CompletableFuture compositeCloseFuture = FutureUtil.waitForAll(closeFutures); + + if (getState() == State.Closing || getState() == State.Closed) { closeConsumerTasks(); failPendingReceive().whenComplete((r, t) -> closeFuture.complete(null)); - return closeFuture; + return compositeCloseFuture; } consumersClosedCounter.increment(); @@ -1114,7 +1143,7 @@ public synchronized CompletableFuture closeAsync() { deregisterFromClientCnx(); client.cleanupConsumer(this); failPendingReceive().whenComplete((r, t) -> closeFuture.complete(null)); - return closeFuture; + return compositeCloseFuture; } stats.getStatTimeout().ifPresent(Timeout::cancel); @@ -1141,23 +1170,7 @@ public synchronized CompletableFuture closeAsync() { }); } - ArrayList> closeFutures = new ArrayList<>(4); - closeFutures.add(closeFuture); - if (retryLetterProducer != null) { - closeFutures.add(retryLetterProducer.thenCompose(p -> p.closeAsync()).whenComplete((ignore, ex) -> { - if (ex != null) { - log.warn("Exception ignored in closing retryLetterProducer of consumer", ex); - } - })); - } - if (deadLetterProducer != null) { - closeFutures.add(deadLetterProducer.thenCompose(p -> p.closeAsync()).whenComplete((ignore, ex) -> { - if (ex != null) { - log.warn("Exception ignored in closing deadLetterProducer of consumer", ex); - } - })); - } - return FutureUtil.waitForAll(closeFutures); + return compositeCloseFuture; } private void cleanupAtClose(CompletableFuture closeFuture, Throwable exception) { @@ -2216,47 +2229,54 @@ private CompletableFuture processPossibleToDLQ(MessageIdAdv messageId) } CompletableFuture result = new CompletableFuture<>(); if (deadLetterMessages != null) { - initDeadLetterProducerIfNeeded(); List> finalDeadLetterMessages = deadLetterMessages; - deadLetterProducer.thenAcceptAsync(producerDLQ -> { + initDeadLetterProducerIfNeeded().thenAcceptAsync(producerDLQ -> { for (MessageImpl message : finalDeadLetterMessages) { - String originMessageIdStr = message.getMessageId().toString(); - String originTopicNameStr = getOriginTopicNameStr(message); - TypedMessageBuilder typedMessageBuilderNew = - producerDLQ.newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) - .value(message.getData()) - .properties(getPropertiesMap(message, originMessageIdStr, originTopicNameStr)); - copyMessageKeysIfNeeded(message, typedMessageBuilderNew); - typedMessageBuilderNew.sendAsync() - .thenAccept(messageIdInDLQ -> { - possibleSendToDeadLetterTopicMessages.remove(messageId); - acknowledgeAsync(messageId).whenComplete((v, ex) -> { - if (ex != null) { - log.warn("[{}] [{}] [{}] Failed to acknowledge the message {} of the original" - + " topic but send to the DLQ successfully.", - topicName, subscription, consumerName, messageId, ex); - result.complete(false); + try { + String originMessageIdStr = message.getMessageId().toString(); + String originTopicNameStr = getOriginTopicNameStr(message); + TypedMessageBuilder typedMessageBuilderNew = + producerDLQ.newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) + .value(message.getData()) + .properties(getPropertiesMap(message, originMessageIdStr, originTopicNameStr)); + copyMessageKeysIfNeeded(message, typedMessageBuilderNew); + typedMessageBuilderNew.sendAsync() + .thenAccept(messageIdInDLQ -> { + possibleSendToDeadLetterTopicMessages.remove(messageId); + acknowledgeAsync(messageId).whenComplete((v, ex) -> { + if (ex != null) { + log.warn( + "[{}] [{}] [{}] Failed to acknowledge the message {} of the " + + "original topic but send to the DLQ successfully.", + topicName, subscription, consumerName, messageId, ex); + result.complete(false); + } else { + result.complete(true); + } + }); + }).exceptionally(ex -> { + if (ex instanceof PulsarClientException.ProducerQueueIsFullError) { + log.warn( + "[{}] [{}] [{}] Failed to send DLQ message to {} for message id {}: {}", + topicName, subscription, consumerName, + deadLetterPolicy.getDeadLetterTopic(), messageId, ex.getMessage()); } else { - result.complete(true); + log.warn("[{}] [{}] [{}] Failed to send DLQ message to {} for message id {}", + topicName, subscription, consumerName, + deadLetterPolicy.getDeadLetterTopic(), messageId, ex); } + result.complete(false); + return null; }); - }).exceptionally(ex -> { - if (ex instanceof PulsarClientException.ProducerQueueIsFullError) { - log.warn("[{}] [{}] [{}] Failed to send DLQ message to {} for message id {}: {}", - topicName, subscription, consumerName, - deadLetterPolicy.getDeadLetterTopic(), messageId, ex.getMessage()); - } else { - log.warn("[{}] [{}] [{}] Failed to send DLQ message to {} for message id {}", - topicName, subscription, consumerName, - deadLetterPolicy.getDeadLetterTopic(), messageId, ex); - } - result.complete(false); - return null; - }); + } catch (Exception e) { + log.warn("[{}] [{}] [{}] Failed to send DLQ message to {} for message id {}", + topicName, subscription, consumerName, deadLetterPolicy.getDeadLetterTopic(), messageId, + e); + result.complete(false); + } } }, internalPinnedExecutor).exceptionally(ex -> { log.error("Dead letter producer exception with topic: {}", deadLetterPolicy.getDeadLetterTopic(), ex); - deadLetterProducer = null; result.complete(false); return null; }); @@ -2266,51 +2286,112 @@ private CompletableFuture processPossibleToDLQ(MessageIdAdv messageId) return result; } - private void initDeadLetterProducerIfNeeded() { - if (deadLetterProducer == null) { + private CompletableFuture> initDeadLetterProducerIfNeeded() { + CompletableFuture> p = deadLetterProducer; + if (p == null || p.isCompletedExceptionally()) { createProducerLock.writeLock().lock(); try { - if (deadLetterProducer == null) { - deadLetterProducer = - ((ProducerBuilderImpl) client.newProducer(Schema.AUTO_PRODUCE_BYTES(schema))) - .initialSubscriptionName(this.deadLetterPolicy.getInitialSubscriptionName()) - .topic(this.deadLetterPolicy.getDeadLetterTopic()) - .producerName(String.format("%s-%s-%s-%s-DLQ", this.topicName, this.subscription, - this.consumerName, RandomStringUtils.randomAlphanumeric(5))) - .blockIfQueueFull(false) - .enableBatching(false) - .enableChunking(true) - .createAsync(); - deadLetterProducer.thenAccept(dlqProducer -> { - stats.setDeadLetterProducerStats(dlqProducer.getStats()); - }); + p = deadLetterProducer; + if (p == null || p.isCompletedExceptionally()) { + p = createProducerWithBackOff(() -> { + CompletableFuture> newProducer = + ((ProducerBuilderImpl) client.newProducer(Schema.AUTO_PRODUCE_BYTES(schema))) + .initialSubscriptionName(this.deadLetterPolicy.getInitialSubscriptionName()) + .topic(this.deadLetterPolicy.getDeadLetterTopic()) + .producerName( + String.format("%s-%s-%s-%s-DLQ", this.topicName, this.subscription, + this.consumerName, RandomStringUtils.randomAlphanumeric(5))) + .blockIfQueueFull(false) + .enableBatching(false) + .enableChunking(true) + .createAsync(); + newProducer.whenComplete((producer, ex) -> { + if (ex != null) { + log.error("[{}] [{}] [{}] Failed to create dead letter producer for topic {}", + topicName, subscription, consumerName, deadLetterPolicy.getDeadLetterTopic(), + ex); + deadLetterProducerFailureCount++; + } else { + deadLetterProducerFailureCount = 0; + stats.setDeadLetterProducerStats(producer.getStats()); + } + }); + return newProducer; + }, deadLetterProducerFailureCount, () -> "dead letter producer (topic: " + + deadLetterPolicy.getDeadLetterTopic() + ")"); + deadLetterProducer = p; } } finally { createProducerLock.writeLock().unlock(); } } + return p; } - private void initRetryLetterProducerIfNeeded() { - if (retryLetterProducer == null) { + private CompletableFuture> createProducerWithBackOff( + Supplier>> producerSupplier, int failureCount, + Supplier logDescription) { + if (failureCount == 0) { + return producerSupplier.get(); + } else { + // calculate backoff time for given failure count + Backoff backoff = new BackoffBuilder() + .setInitialTime(100, TimeUnit.MILLISECONDS) + .setMandatoryStop(client.getConfiguration().getOperationTimeoutMs() * 2, + TimeUnit.MILLISECONDS) + .setMax(1, TimeUnit.MINUTES) + .create(); + long backoffTimeMillis = 0; + for (int i = 0; i < failureCount; i++) { + backoffTimeMillis = backoff.next(); + } + CompletableFuture> newProducer = new CompletableFuture<>(); + ScheduledExecutorService executor = + (ScheduledExecutorService) client.getScheduledExecutorProvider().getExecutor(this); + log.info("Creating {} with backoff time of {} ms", logDescription.get(), backoffTimeMillis); + executor.schedule(() -> { + FutureUtil.completeAfter(newProducer, producerSupplier.get()); + }, backoffTimeMillis, TimeUnit.MILLISECONDS); + return newProducer; + } + } + + private CompletableFuture> initRetryLetterProducerIfNeeded() { + CompletableFuture> p = retryLetterProducer; + if (p == null || p.isCompletedExceptionally()) { createProducerLock.writeLock().lock(); try { - if (retryLetterProducer == null) { - retryLetterProducer = client - .newProducer(Schema.AUTO_PRODUCE_BYTES(schema)) - .topic(this.deadLetterPolicy.getRetryLetterTopic()) - .enableBatching(false) - .enableChunking(true) - .blockIfQueueFull(false) - .createAsync(); - retryLetterProducer.thenAccept(rtlProducer -> { - stats.setRetryLetterProducerStats(rtlProducer.getStats()); - }); + p = retryLetterProducer; + if (p == null || p.isCompletedExceptionally()) { + p = createProducerWithBackOff(() -> { + CompletableFuture> newProducer = client + .newProducer(Schema.AUTO_PRODUCE_BYTES(schema)) + .topic(this.deadLetterPolicy.getRetryLetterTopic()) + .enableBatching(false) + .enableChunking(true) + .blockIfQueueFull(false) + .createAsync(); + newProducer.whenComplete((producer, ex) -> { + if (ex != null) { + log.error("[{}] [{}] [{}] Failed to create retry letter producer for topic {}", + topicName, subscription, consumerName, deadLetterPolicy.getRetryLetterTopic(), + ex); + retryLetterProducerFailureCount++; + } else { + retryLetterProducerFailureCount = 0; + stats.setRetryLetterProducerStats(producer.getStats()); + } + }); + return newProducer; + }, retryLetterProducerFailureCount, () -> "retry letter producer (topic: " + + deadLetterPolicy.getRetryLetterTopic() + ")"); + retryLetterProducer = p; } } finally { createProducerLock.writeLock().unlock(); } } + return p; } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 6f9c5b47c55bb..341272cd69bf8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -638,7 +638,14 @@ public CompletableFuture closeAsync() { CompletableFuture closeFuture = new CompletableFuture<>(); List> futureList = consumers.values().stream() - .map(ConsumerImpl::closeAsync).collect(Collectors.toList()); + .map(consumer -> consumer.closeAsync().exceptionally(t -> { + Throwable cause = FutureUtil.unwrapCompletionException(t); + if (!(cause instanceof PulsarClientException.AlreadyClosedException)) { + log.warn("[{}] [{}] Error closing individual consumer", consumer.getTopic(), + consumer.getSubscription(), cause); + } + return null; + })).collect(Collectors.toList()); FutureUtil.waitForAll(futureList) .thenComposeAsync((r) -> { diff --git a/pulsar-client/src/main/resources/findbugsExclude.xml b/pulsar-client/src/main/resources/findbugsExclude.xml index 0e05d20cb9bb4..f7cf6b9cfd50e 100644 --- a/pulsar-client/src/main/resources/findbugsExclude.xml +++ b/pulsar-client/src/main/resources/findbugsExclude.xml @@ -1043,4 +1043,8 @@ + + + + From dc6397093eb645fec1bfd6f127f78ad36be0a360 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 13 Jan 2025 10:39:02 +0800 Subject: [PATCH 246/327] [fix][broker] Remove blocking calls from internalGetPartitionedStats (#23832) Signed-off-by: Zixuan Liu --- .../admin/impl/PersistentTopicsBase.java | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index eed667d499000..e88b1110d0ad7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1522,23 +1522,21 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean } } if (perPartition && stats.partitions.isEmpty()) { - try { - boolean pathExists = namespaceResources().getPartitionedTopicResources() - .partitionedTopicExists(topicName); - if (pathExists) { - stats.partitions.put(topicName.toString(), new TopicStatsImpl()); - } else { - asyncResponse.resume( - new RestException(Status.NOT_FOUND, - "Internal topics have not been generated yet")); - return null; - } - } catch (Exception e) { - asyncResponse.resume(new RestException(e)); - return null; - } + namespaceResources().getPartitionedTopicResources() + .partitionedTopicExistsAsync(topicName) + .thenAccept(exists -> { + if (exists) { + stats.partitions.put(topicName.toString(), new TopicStatsImpl()); + asyncResponse.resume(stats); + } else { + asyncResponse.resume( + new RestException(Status.NOT_FOUND, + "Internal topics have not been generated yet")); + } + }); + } else { + asyncResponse.resume(stats); } - asyncResponse.resume(stats); return null; }); }).exceptionally(ex -> { From d3707c56db691304f4a1caceb559d3f29508fa6f Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 13 Jan 2025 12:06:36 +0800 Subject: [PATCH 247/327] [improve][broker] Reduce unnecessary REPLICATED_SUBSCRIPTION_SNAPSHOT_REQUEST (#23839) --- .../ReplicatedSubscriptionsController.java | 23 +++++ .../service/ReplicatedSubscriptionTest.java | 90 +++++++++++++++++++ 2 files changed, 113 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java index 4fb0022194a02..b21fe7acfdb6f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service.persistent; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; @@ -220,6 +221,23 @@ private void receiveSubscriptionUpdated(ReplicatedSubscriptionsUpdate update) { private void startNewSnapshot() { cleanupTimedOutSnapshots(); + if (lastCompletedSnapshotStartTime == 0 && !pendingSnapshots.isEmpty()) { + // 1. If the remote cluster has disabled subscription replication or there's an incorrect config, + // it will not respond to SNAPSHOT_REQUEST. Therefore, lastCompletedSnapshotStartTime will remain 0, + // making it unnecessary to resend the request. + // 2. This approach prevents sending additional SNAPSHOT_REQUEST to both local_topic and remote_topic. + // 3. Since it's uncertain when the remote cluster will enable subscription replication, + // the timeout mechanism of pendingSnapshots is used to ensure retries. + // + // In other words, when hit this case, The frequency of sending SNAPSHOT_REQUEST + // will use `replicatedSubscriptionsSnapshotTimeoutSeconds`. + if (log.isDebugEnabled()) { + log.debug("[{}] PendingSnapshot exists but has never succeeded. " + + "Skipping snapshot creation until pending snapshot timeout.", topic.getName()); + } + return; + } + if (topic.getLastMaxReadPositionMovedForwardTimestamp() < lastCompletedSnapshotStartTime || topic.getLastMaxReadPositionMovedForwardTimestamp() == 0) { // There was no message written since the last snapshot, we can skip creating a new snapshot @@ -324,6 +342,11 @@ String localCluster() { return localCluster; } + @VisibleForTesting + public ConcurrentMap pendingSnapshots() { + return pendingSnapshots; + } + @Override public boolean isMarkerMessage() { // Everything published by this controller will be a marker a message diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java index 5b896a22baa33..0f527993bba59 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java @@ -64,6 +64,10 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; +import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.common.api.proto.MarkerType; +import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.policies.data.PartitionedTopicStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -1002,6 +1006,92 @@ public void testReplicatedSubscriptionWithCompaction() throws Exception { Assert.assertEquals(result, List.of("V2")); } + @Test + public void testReplicatedSubscriptionOneWay() throws Exception { + final String namespace = BrokerTestUtil.newUniqueName("pulsar-r4/replicatedsubscription"); + final String topicName = "persistent://" + namespace + "/one-way"; + int defaultSubscriptionsSnapshotFrequency = config1.getReplicatedSubscriptionsSnapshotFrequencyMillis(); + int defaultSubscriptionsSnapshotTimeout = config1.getReplicatedSubscriptionsSnapshotTimeoutSeconds(); + config1.setReplicatedSubscriptionsSnapshotTimeoutSeconds(2); + config1.setReplicatedSubscriptionsSnapshotFrequencyMillis(100); + + // cluster4 disabled ReplicatedSubscriptions + admin1.tenants().createTenant("pulsar-r4", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid4"), Sets.newHashSet(cluster1, cluster4))); + admin1.namespaces().createNamespace(namespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet(cluster1, cluster4)); + + String subscriptionName = "cluster-subscription"; + boolean replicateSubscriptionState = true; + + @Cleanup + PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + + @Cleanup + final PulsarClient client4 = PulsarClient.builder().serviceUrl(url4.toString()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + + // create subscription in cluster4 + createReplicatedSubscription(client1, topicName, subscriptionName, replicateSubscriptionState); + // create subscription in cluster4 + createReplicatedSubscription(client4, topicName, subscriptionName, replicateSubscriptionState); + + // send messages in cluster1 + @Cleanup + Producer producer = client1.newProducer().topic(topicName) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + int numMessages = 6; + for (int i = 0; i < numMessages; i++) { + String body = "message" + i; + producer.send(body.getBytes(StandardCharsets.UTF_8)); + } + producer.close(); + + // wait for snapshot marker request to be replicated + Thread.sleep(3 * config1.getReplicatedSubscriptionsSnapshotFrequencyMillis()); + + // Assert just have 1 pending snapshot in cluster1 + final PersistentTopic topic1 = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + ReplicatedSubscriptionsController r1Controller = + topic1.getReplicatedSubscriptionController().get(); + assertEquals(r1Controller.pendingSnapshots().size(), 1); + + // Assert cluster4 just receive 1 snapshot request msg + int numSnapshotRequest = 0; + List> r4Messages = admin4.topics() + .peekMessages(topicName, subscriptionName, 100, true, TransactionIsolationLevel.READ_UNCOMMITTED); + for (Message r4Message : r4Messages) { + MessageMetadata msgMetadata = ((MessageImpl) r4Message).getMessageBuilder(); + if (msgMetadata.hasMarkerType() && msgMetadata.getMarkerType() == MarkerType.REPLICATED_SUBSCRIPTION_SNAPSHOT_REQUEST_VALUE) { + numSnapshotRequest++; + } + } + Assert.assertEquals(numSnapshotRequest, 1); + + // Wait pending snapshot timeout + Thread.sleep(config1.getReplicatedSubscriptionsSnapshotTimeoutSeconds() * 1000); + numSnapshotRequest = 0; + r4Messages = admin4.topics() + .peekMessages(topicName, subscriptionName, 100, true, TransactionIsolationLevel.READ_UNCOMMITTED); + for (Message r4Message : r4Messages) { + MessageMetadata msgMetadata = ((MessageImpl) r4Message).getMessageBuilder(); + if (msgMetadata.hasMarkerType() && msgMetadata.getMarkerType() == MarkerType.REPLICATED_SUBSCRIPTION_SNAPSHOT_REQUEST_VALUE) { + numSnapshotRequest++; + } + } + Assert.assertEquals(numSnapshotRequest, 2); + + // Set back to default config. + config1.setReplicatedSubscriptionsSnapshotTimeoutSeconds(defaultSubscriptionsSnapshotTimeout); + config1.setReplicatedSubscriptionsSnapshotFrequencyMillis(defaultSubscriptionsSnapshotFrequency); + } + /** * Disable replication subscription. * Test scheduled task case. From ba04a43b1b0b726080612361b3da8355f73ee70a Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Mon, 13 Jan 2025 14:14:32 +0800 Subject: [PATCH 248/327] [fix][pip] PIP-399: Fix Metric Name for Delayed Queue (#23789) --- pip/pip-399.md | 60 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 60 insertions(+) create mode 100644 pip/pip-399.md diff --git a/pip/pip-399.md b/pip/pip-399.md new file mode 100644 index 0000000000000..0db472617608c --- /dev/null +++ b/pip/pip-399.md @@ -0,0 +1,60 @@ + +# PIP-399: Fix Metric Name for Delayed Queue + +# Background knowledge + +Pulsar delayed delivery is a feature that allows messages to be delivered to consumers after a certain delay. +It will expose a metric to monitor the memory usage of delayed queue. + +# Motivation + +There is already one metric called `pulsar_delayed_message_index_size_bytes` for the total memory occupation used by delayed queue of **one topic**. +``` +writeMetric(stream, "pulsar_delayed_message_index_size_bytes", stats.delayedTrackerMemoryUsage, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); +``` + +Whereas, the metric for **one sub** also called `pulsar_delayed_message_index_size_bytes`, which do not comform the metric name norm and is confusing. +``` +writeSubscriptionMetric(stream, "pulsar_delayed_message_index_size_bytes", + subsStats.delayedTrackerMemoryUsage, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); +``` + +Currently, it can export metric like: +``` +# TYPE pulsar_delayed_message_index_size_bytes gauge +pulsar_delayed_message_index_size_bytes{cluster="MyPulsar",namespace="public/default",topic="persistent://public/default/testNack-partition-0"} 0 +pulsar_delayed_message_index_size_bytes{cluster="MyPulsar",namespace="public/default",topic="persistent://public/default/testNack-partition-0",subscription="sub2"} 0 +``` + +The metric of topic and subscription mix together. If we want to filter out the metric of sub to pick out the metric of topic, we need to use promsql like: +`pulsar_delayed_message_index_size_bytes{subscription=""}` +It is quite weird and not friendly to use. + + +# Goals + +Rename the metric for **one sub** to `pulsar_subscription_delayed_message_index_size_bytes`. + + +# Backward & Forward Compatibility + +## Upgrade + +Rename the metric for **one sub** to `pulsar_subscription_delayed_message_index_size_bytes`. + +## Downgrade / Rollback + +Rename the metric for **one sub** to `pulsar_delayed_message_index_size_bytes`. + + + +# General Notes + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/b8rqld3cww1t34zntgmld50yz34lxx1d +* Mailing List voting thread: https://lists.apache.org/thread/cyyx29ggjdpbr3kq5vvd6tk83f9vc112 From 999db2c7f5a8a3c09e2055fb47f9e3cb65e7d1d2 Mon Sep 17 00:00:00 2001 From: feynmanlin <315157973@qq.com> Date: Tue, 14 Jan 2025 10:41:59 +0800 Subject: [PATCH 249/327] [fix] [broker] Fix acknowledgeCumulativeAsync block when ackReceipt is enabled (#23841) --- .../pulsar/client/impl/ConsumerAckTest.java | 34 +++++++++++++++++++ ...sistentAcknowledgmentsGroupingTracker.java | 7 +++- 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java index a83283bc267b5..6d9025fd966b3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java @@ -116,6 +116,40 @@ public void testAckResponse() throws PulsarClientException, InterruptedException Assert.assertTrue(e.getCause() instanceof PulsarClientException.NotAllowedException); } } + @Test(timeOut = 30000) + public void testAckReceipt() throws Exception { + String topic = "testAckReceipt"; + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .enableBatching(false) + .create(); + @Cleanup + ConsumerImpl consumer = (ConsumerImpl) pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .isAckReceiptEnabled(true) + .subscribe(); + for (int i = 0; i < 10; i++) { + producer.send(i); + } + Message message = consumer.receive(); + MessageId messageId = message.getMessageId(); + consumer.acknowledgeCumulativeAsync(messageId).get(); + consumer.acknowledgeCumulativeAsync(messageId).get(); + consumer.close(); + @Cleanup + ConsumerImpl consumer2 = (ConsumerImpl) pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .isAckReceiptEnabled(true) + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscribe(); + message = consumer2.receive(); + messageId = message.getMessageId(); + consumer2.acknowledgeCumulativeAsync(messageId).get(); + consumer2.acknowledgeCumulativeAsync(messageId).get(); + } @Test public void testIndividualAck() throws Exception { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index c0ee13b346a0b..d30c3de0fd720 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -312,7 +312,12 @@ private CompletableFuture doCumulativeAck(MessageIdAdv messageId, Map readLock = acquireReadLock(); try { doCumulativeAckAsync(messageId, bitSet); - return readLock.map(__ -> currentCumulativeAckFuture).orElse(CompletableFuture.completedFuture(null)); + return readLock.map(__ -> { + if (consumer.isAckReceiptEnabled() && lastCumulativeAck.compareTo(messageId) == 0) { + return CompletableFuture.completedFuture(null); + } + return currentCumulativeAckFuture; + }).orElse(CompletableFuture.completedFuture(null)); } finally { readLock.ifPresent(Lock::unlock); } From f1f65a52eca65fd75171824c32be6b7597f0bc4c Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 14 Jan 2025 16:30:44 +0800 Subject: [PATCH 250/327] [fix][broker] Continue using the next provider for http authentication if one fails (#23842) Signed-off-by: Zixuan Liu --- .../authentication/AuthenticationService.java | 22 +-- .../auth/AuthenticationServiceTest.java | 126 ++++++++++++++++++ 2 files changed, 133 insertions(+), 15 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java index f6eb785d2e479..e2bf4dcc0156d 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java @@ -24,11 +24,10 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import javax.naming.AuthenticationException; import javax.servlet.http.HttpServletRequest; @@ -49,7 +48,7 @@ public class AuthenticationService implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(AuthenticationService.class); private final String anonymousUserRole; - private final Map providers = new HashMap<>(); + private final Map providers = new LinkedHashMap<>(); public AuthenticationService(ServiceConfiguration conf) throws PulsarServerException { this(conf, OpenTelemetry.noop()); @@ -60,7 +59,7 @@ public AuthenticationService(ServiceConfiguration conf, OpenTelemetry openTeleme anonymousUserRole = conf.getAnonymousUserRole(); if (conf.isAuthenticationEnabled()) { try { - Map> providerMap = new HashMap<>(); + Map> providerMap = new LinkedHashMap<>(); for (String className : conf.getAuthenticationProviders()) { if (className.isEmpty()) { continue; @@ -131,7 +130,7 @@ public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletRe AuthenticationProvider providerToUse = getAuthProvider(authMethodName); try { return providerToUse.authenticateHttpRequest(request, response); - } catch (AuthenticationException e) { + } catch (Exception e) { if (LOG.isDebugEnabled()) { LOG.debug("Authentication failed for provider " + providerToUse.getAuthMethodName() + " : " + e.getMessage(), e); @@ -142,7 +141,7 @@ public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletRe for (AuthenticationProvider provider : providers.values()) { try { return provider.authenticateHttpRequest(request, response); - } catch (AuthenticationException e) { + } catch (Exception e) { if (LOG.isDebugEnabled()) { LOG.debug("Authentication failed for provider " + provider.getAuthMethodName() + ": " + e.getMessage(), e); @@ -183,25 +182,18 @@ public String authenticateHttpRequest(HttpServletRequest request, Authentication } // Backward compatible, the authData value was null in the previous implementation return providerToUse.authenticateAsync(authData).get(); - } catch (AuthenticationException e) { + } catch (Exception e) { if (LOG.isDebugEnabled()) { LOG.debug("Authentication failed for provider " + providerToUse.getAuthMethodName() + " : " + e.getMessage(), e); } - throw e; - } catch (ExecutionException | InterruptedException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Authentication failed for provider " + providerToUse.getAuthMethodName() + " : " - + e.getMessage(), e); - } - throw new RuntimeException(e); } } else { for (AuthenticationProvider provider : providers.values()) { try { AuthenticationState authenticationState = provider.newHttpAuthState(request); return provider.authenticateAsync(authenticationState.getAuthDataSource()).get(); - } catch (ExecutionException | InterruptedException | AuthenticationException e) { + } catch (Exception e) { if (LOG.isDebugEnabled()) { LOG.debug("Authentication failed for provider " + provider.getAuthMethodName() + ": " + e.getMessage(), e); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java index 78ae046b0c8c8..0d5fc4e19d5b3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java @@ -20,6 +20,8 @@ import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedDataAttributeName; import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedRoleAttributeName; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; @@ -29,15 +31,22 @@ import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; import java.io.IOException; +import java.util.LinkedHashSet; import java.util.Set; +import java.util.concurrent.CompletableFuture; import javax.naming.AuthenticationException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import lombok.Cleanup; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationDataCommand; import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationProvider; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.broker.authentication.AuthenticationState; +import org.apache.pulsar.broker.web.AuthenticationFilter; +import org.apache.pulsar.common.api.AuthData; import org.testng.Assert; import org.testng.annotations.Test; @@ -166,6 +175,123 @@ public void testAuthenticationHttpRequestResponseWithAnonymousRole() throws Exce service.close(); } + @Test + public void testHttpRequestWithMultipleProviders() throws Exception { + ServiceConfiguration config = new ServiceConfiguration(); + Set providersClassNames = new LinkedHashSet<>(); + providersClassNames.add(MockAuthenticationProviderAlwaysFail.class.getName()); + providersClassNames.add(MockHttpAuthenticationProvider.class.getName()); + config.setAuthenticationProviders(providersClassNames); + config.setAuthenticationEnabled(true); + @Cleanup + AuthenticationService service = new AuthenticationService(config); + + HttpServletRequest request = mock(HttpServletRequest.class); + + when(request.getParameter("role")).thenReturn("success-role1"); + assertTrue(service.authenticateHttpRequest(request, (HttpServletResponse) null)); + + when(request.getParameter("role")).thenReturn(""); + assertThatThrownBy(() -> service.authenticateHttpRequest(request, (HttpServletResponse) null)) + .isInstanceOf(AuthenticationException.class); + + when(request.getParameter("role")).thenReturn("error-role1"); + assertThatThrownBy(() -> service.authenticateHttpRequest(request, (HttpServletResponse) null)) + .isInstanceOf(AuthenticationException.class); + + when(request.getHeader(AuthenticationFilter.PULSAR_AUTH_METHOD_NAME)).thenReturn("http-auth"); + assertThatThrownBy(() -> service.authenticateHttpRequest(request, (HttpServletResponse) null)) + .isInstanceOf(RuntimeException.class); + + HttpServletRequest requestForAuthenticationDataSource = mock(HttpServletRequest.class); + assertThatThrownBy(() -> service.authenticateHttpRequest(requestForAuthenticationDataSource, + (AuthenticationDataSource) null)) + .isInstanceOf(AuthenticationException.class); + + when(requestForAuthenticationDataSource.getParameter("role")).thenReturn("error-role2"); + assertThatThrownBy(() -> service.authenticateHttpRequest(requestForAuthenticationDataSource, + (AuthenticationDataSource) null)) + .isInstanceOf(AuthenticationException.class); + + when(requestForAuthenticationDataSource.getParameter("role")).thenReturn("success-role2"); + assertThat(service.authenticateHttpRequest(requestForAuthenticationDataSource, + (AuthenticationDataSource) null)).isEqualTo("role2"); + } + + public static class MockHttpAuthenticationProvider implements AuthenticationProvider { + @Override + public void close() throws IOException { + } + + @Override + public void initialize(ServiceConfiguration config) throws IOException { + } + + @Override + public String getAuthMethodName() { + return "http-auth"; + } + + private String getRole(HttpServletRequest request) { + String role = request.getParameter("role"); + if (role != null) { + String[] s = role.split("-"); + if (s.length == 2 && s[0].equals("success")) { + return s[1]; + } + } + return null; + } + + @Override + public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletResponse response) { + String role = getRole(request); + if (role != null) { + return true; + } + throw new RuntimeException("test authentication failed"); + } + + @Override + public String authenticate(AuthenticationDataSource authData) throws AuthenticationException { + return authData.getCommandData(); + } + + @Override + public AuthenticationState newHttpAuthState(HttpServletRequest request) throws AuthenticationException { + String role = getRole(request); + if (role != null) { + return new AuthenticationState() { + @Override + public String getAuthRole() throws AuthenticationException { + return role; + } + + @Override + public AuthData authenticate(AuthData authData) throws AuthenticationException { + return null; + } + + @Override + public AuthenticationDataSource getAuthDataSource() { + return new AuthenticationDataCommand(role); + } + + @Override + public boolean isComplete() { + return true; + } + + @Override + public CompletableFuture authenticateAsync(AuthData authData) { + return AuthenticationState.super.authenticateAsync(authData); + } + }; + } + throw new RuntimeException("new http auth failed"); + } + } + public static class MockAuthenticationProvider implements AuthenticationProvider { @Override From c92930f544119d80e3451a5027694d56a2662ff2 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 14 Jan 2025 16:58:11 +0800 Subject: [PATCH 251/327] [fix][broker] Fix possible mark delete NPE when batch index ack is enabled (#23833) --- .../bookkeeper/mledger/impl/AckSetState.java | 4 +- .../mledger/impl/ManagedCursorImpl.java | 143 ++++++++---------- .../PendingAckInMemoryDeleteTest.java | 7 +- .../impl/MLPendingAckStoreTest.java | 3 +- 4 files changed, 65 insertions(+), 92 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetState.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetState.java index 363336e83113e..6e9bf78b14cfe 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetState.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetState.java @@ -18,6 +18,8 @@ */ package org.apache.bookkeeper.mledger.impl; +import javax.annotation.Nullable; + /** * Interface to manage the ackSet state attached to a position. * Helpers in {@link AckSetStateUtil} to create positions with @@ -28,7 +30,7 @@ public interface AckSetState { * Get the ackSet bitset information encoded as a long array. * @return the ackSet */ - long[] getAckSet(); + @Nullable long[] getAckSet(); /** * Set the ackSet bitset information as a long array. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 50f5f36b2d53d..12cb22c161a90 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -35,13 +35,14 @@ import java.time.Clock; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -61,6 +62,8 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.LongStream; +import javax.annotation.Nullable; +import lombok.Getter; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; import org.apache.bookkeeper.client.BKException; @@ -105,7 +108,6 @@ import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer; import org.apache.pulsar.common.util.collections.LongPairRangeSet.RangeBoundConsumer; @@ -200,7 +202,9 @@ public class ManagedCursorImpl implements ManagedCursor { // Maintain the deletion status for batch messages // (ledgerId, entryId) -> deletion indexes - protected final ConcurrentSkipListMap batchDeletedIndexes; + @Getter + @VisibleForTesting + @Nullable protected final ConcurrentSkipListMap batchDeletedIndexes; private final ReadWriteLock lock = new ReentrantReadWriteLock(); private RateLimiter markDeleteLimiter; @@ -709,6 +713,7 @@ private void recoverIndividualDeletedMessages(List i private void recoverBatchDeletedIndexes ( List batchDeletedIndexInfoList) { + Objects.requireNonNull(batchDeletedIndexes); lock.writeLock().lock(); try { this.batchDeletedIndexes.clear(); @@ -720,8 +725,7 @@ private void recoverBatchDeletedIndexes ( } this.batchDeletedIndexes.put( PositionFactory.create(batchDeletedIndexInfo.getPosition().getLedgerId(), - batchDeletedIndexInfo.getPosition().getEntryId()), - BitSetRecyclable.create().resetWords(array)); + batchDeletedIndexInfo.getPosition().getEntryId()), BitSet.valueOf(array)); } }); } finally { @@ -1409,14 +1413,12 @@ public void operationComplete() { lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, isCompactionCursor() ? getProperties() : Collections.emptyMap(), null, null); individualDeletedMessages.clear(); - if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - batchDeletedIndexes.values().forEach(BitSetRecyclable::recycle); + if (batchDeletedIndexes != null) { batchDeletedIndexes.clear(); AckSetStateUtil.maybeGetAckSetState(newReadPosition).ifPresent(ackSetState -> { long[] resetWords = ackSetState.getAckSet(); if (resetWords != null) { - BitSetRecyclable ackSet = BitSetRecyclable.create().resetWords(resetWords); - batchDeletedIndexes.put(newReadPosition, ackSet); + batchDeletedIndexes.put(newReadPosition, BitSet.valueOf(resetWords)); } }); } @@ -2045,47 +2047,7 @@ public void asyncMarkDelete(final Position position, Map propertie log.debug("[{}] Mark delete cursor {} up to position: {}", ledger.getName(), name, position); } - Position newPosition = position; - - Optional ackSetStateOptional = AckSetStateUtil.maybeGetAckSetState(newPosition); - if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - if (ackSetStateOptional.isPresent()) { - AtomicReference bitSetRecyclable = new AtomicReference<>(); - BitSetRecyclable givenBitSet = - BitSetRecyclable.create().resetWords(ackSetStateOptional.map(AckSetState::getAckSet).get()); - // In order to prevent the batch index recorded in batchDeletedIndexes from rolling back, - // only update batchDeletedIndexes when the submitted batch index is greater - // than the recorded index. - batchDeletedIndexes.compute(newPosition, - (k, v) -> { - if (v == null) { - return givenBitSet; - } - if (givenBitSet.nextSetBit(0) > v.nextSetBit(0)) { - bitSetRecyclable.set(v); - return givenBitSet; - } else { - bitSetRecyclable.set(givenBitSet); - return v; - } - }); - if (bitSetRecyclable.get() != null) { - bitSetRecyclable.get().recycle(); - } - newPosition = ledger.getPreviousPosition(newPosition); - } - Map subMap = batchDeletedIndexes.subMap(PositionFactory.EARLIEST, newPosition); - subMap.values().forEach(BitSetRecyclable::recycle); - subMap.clear(); - } else { - if (ackSetStateOptional.isPresent()) { - AckSetState ackSetState = ackSetStateOptional.get(); - if (ackSetState.getAckSet() != null) { - newPosition = ledger.getPreviousPosition(newPosition); - } - } - } - + Position newPosition = ackBatchPosition(position); if (ledger.getLastConfirmedEntry().compareTo(newPosition) < 0) { boolean shouldCursorMoveForward = false; try { @@ -2131,6 +2093,31 @@ public void asyncMarkDelete(final Position position, Map propertie internalAsyncMarkDelete(newPosition, properties, callback, ctx); } + private Position ackBatchPosition(Position position) { + return AckSetStateUtil.maybeGetAckSetState(position) + .map(AckSetState::getAckSet) + .map(ackSet -> { + if (batchDeletedIndexes == null) { + return ledger.getPreviousPosition(position); + } + // In order to prevent the batch index recorded in batchDeletedIndexes from rolling back, + // only update batchDeletedIndexes when the submitted batch index is greater + // than the recorded index. + final var givenBitSet = BitSet.valueOf(ackSet); + batchDeletedIndexes.compute(position, (k, v) -> { + if (v == null || givenBitSet.nextSetBit(0) > v.nextSetBit(0)) { + return givenBitSet; + } else { + return v; + } + }); + final var newPosition = ledger.getPreviousPosition(position); + batchDeletedIndexes.subMap(PositionFactory.EARLIEST, newPosition).clear(); + return newPosition; + }) + .orElse(position); + } + protected void internalAsyncMarkDelete(final Position newPosition, Map properties, final MarkDeleteCallback callback, final Object ctx) { ledger.mbean.addMarkDeleteOp(); @@ -2236,12 +2223,10 @@ public void operationComplete() { try { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); - if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - Map subMap = batchDeletedIndexes.subMap(PositionFactory.EARLIEST, + if (batchDeletedIndexes != null) { + batchDeletedIndexes.subMap(PositionFactory.EARLIEST, false, PositionFactory.create(mdEntry.newPosition.getLedgerId(), - mdEntry.newPosition.getEntryId()), true); - subMap.values().forEach(BitSetRecyclable::recycle); - subMap.clear(); + mdEntry.newPosition.getEntryId()), true).clear(); } persistentMarkDeletePosition = mdEntry.newPosition; } finally { @@ -2376,11 +2361,8 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } if (internalIsMessageDeleted(position)) { - if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); - if (bitSetRecyclable != null) { - bitSetRecyclable.recycle(); - } + if (batchDeletedIndexes != null) { + batchDeletedIndexes.remove(position); } if (log.isDebugEnabled()) { log.debug("[{}] [{}] Position was already deleted {}", ledger.getName(), name, position); @@ -2389,11 +2371,8 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } long[] ackSet = AckSetStateUtil.getAckSetArrayOrNull(position); if (ackSet == null) { - if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); - if (bitSetRecyclable != null) { - bitSetRecyclable.recycle(); - } + if (batchDeletedIndexes != null) { + batchDeletedIndexes.remove(position); } // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will // make the RangeSet recognize the "continuity" between adjacent Positions. @@ -2406,12 +2385,11 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name, individualDeletedMessages); } - } else if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(ackSet); - BitSetRecyclable bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> givenBitSet); + } else if (batchDeletedIndexes != null) { + final var givenBitSet = BitSet.valueOf(ackSet); + final var bitSet = batchDeletedIndexes.computeIfAbsent(position, __ -> givenBitSet); if (givenBitSet != bitSet) { bitSet.and(givenBitSet); - givenBitSet.recycle(); } if (bitSet.isEmpty()) { Position previousPosition = ledger.getPreviousPosition(position); @@ -2419,10 +2397,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this); - BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); - if (bitSetRecyclable != null) { - bitSetRecyclable.recycle(); - } + batchDeletedIndexes.remove(position); } } } @@ -3213,7 +3188,7 @@ private List buildIndividualDeletedMessageRanges() { private List buildBatchEntryDeletionIndexInfoList() { lock.readLock().lock(); try { - if (!getConfig().isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { + if (batchDeletedIndexes == null || batchDeletedIndexes.isEmpty()) { return Collections.emptyList(); } MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo @@ -3221,9 +3196,9 @@ private List buildBatchEntryDeletio MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats .BatchedEntryDeletionIndexInfo.newBuilder(); List result = new ArrayList<>(); - Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + final var iterator = batchDeletedIndexes.entrySet().iterator(); while (iterator.hasNext() && result.size() < getConfig().getMaxBatchDeletedIndexToPersist()) { - Map.Entry entry = iterator.next(); + final var entry = iterator.next(); nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); @@ -3643,11 +3618,11 @@ private boolean internalIsMessageDeleted(Position position) { @Override public long[] getBatchPositionAckSet(Position position) { if (batchDeletedIndexes != null) { - BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.get(position); - if (bitSetRecyclable == null) { + final var bitSet = batchDeletedIndexes.get(position); + if (bitSet == null) { return null; } else { - return bitSetRecyclable.toLongArray(); + return bitSet.toLongArray(); } } else { return null; @@ -3750,8 +3725,8 @@ private ManagedCursorImpl cursorImpl() { @Override public long[] getDeletedBatchIndexesAsLongArray(Position position) { - if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - BitSetRecyclable bitSet = batchDeletedIndexes.get(position); + if (batchDeletedIndexes != null) { + final var bitSet = batchDeletedIndexes.get(position); return bitSet == null ? null : bitSet.toLongArray(); } else { return null; @@ -3879,9 +3854,9 @@ public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) thro lock.readLock().unlock(); } if (batchDeletedIndexes != null) { - for (Map.Entry entry : this.batchDeletedIndexes.entrySet()) { - BitSetRecyclable copiedBitSet = BitSetRecyclable.valueOf(entry.getValue()); - newNonDurableCursor.batchDeletedIndexes.put(entry.getKey(), copiedBitSet); + Objects.requireNonNull(newNonDurableCursor.batchDeletedIndexes); + for (final var entry : this.batchDeletedIndexes.entrySet()) { + newNonDurableCursor.batchDeletedIndexes.put(entry.getKey(), (BitSet) entry.getValue().clone()); } } return newNonDurableCursor; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java index 58cf59aa6b3b9..1b4fd451c104f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java @@ -25,7 +25,6 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; @@ -45,7 +44,6 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.api.transaction.TxnID; -import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -223,10 +221,7 @@ public void txnAckTestBatchAndSharedSubMemoryDeleteTest() throws Exception { (LinkedMap>) field.get(pendingAckHandle); assertTrue(individualAckOfTransaction.isEmpty()); managedCursor = (ManagedCursorImpl) testPersistentSubscription.getCursor(); - field = ManagedCursorImpl.class.getDeclaredField("batchDeletedIndexes"); - field.setAccessible(true); - final ConcurrentSkipListMap batchDeletedIndexes = - (ConcurrentSkipListMap) field.get(managedCursor); + final var batchDeletedIndexes = managedCursor.getBatchDeletedIndexes(); if (retryCnt == 0) { //one message are not ack Awaitility.await().until(() -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java index 6dd3e6e7c7822..6bbfb25ee2ff7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java @@ -73,6 +73,7 @@ public class MLPendingAckStoreTest extends TransactionTestBase { @BeforeClass @Override protected void setup() throws Exception { + conf.setAcknowledgmentAtBatchIndexLevelEnabled(true); setUpBase(1, 1, NAMESPACE1 + "/test", 0); } @@ -304,4 +305,4 @@ private LinkedHashSet calculatePendingAckIndexes(List positionList, } return indexes; } -} \ No newline at end of file +} From 5be922b3fd47fb01006bd14f7396321470ef40f2 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Tue, 14 Jan 2025 17:40:15 +0800 Subject: [PATCH 252/327] [fix][broker] PIP-399: Fix Metric Name for Delayed Queue (#23712) --- .../broker/stats/prometheus/TopicStats.java | 2 +- .../persistent/PersistentTopicTest.java | 23 +++++++++++++++---- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index e54a3710e1294..524d47e7c1b92 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -360,7 +360,7 @@ public static void printTopicStats(PrometheusMetricStreams stream, TopicStats st writeSubscriptionMetric(stream, "pulsar_subscription_filter_rescheduled_msg_count", subsStats.filterRescheduledMsgCount, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); - writeSubscriptionMetric(stream, "pulsar_delayed_message_index_size_bytes", + writeSubscriptionMetric(stream, "pulsar_subscription_delayed_message_index_size_bytes", subsStats.delayedMessageIndexSizeInBytes, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index 903443d37bb07..1196161711224 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -424,10 +424,11 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex @Cleanup Producer producer = client.newProducer(Schema.STRING).topic(topic).enableBatching(false).create(); + String subName = "test_sub"; @Cleanup Consumer consumer = client.newConsumer(Schema.STRING) .topic(topic) - .subscriptionName("test_sub") + .subscriptionName(subName) .subscriptionType(SubscriptionType.Shared) .messageListener((MessageListener) (consumer1, msg) -> { try { @@ -453,7 +454,13 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex Multimap metricsMap = parseMetrics(metricsStr); Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes"); - Assert.assertTrue(metrics.size() > 0); + Collection subMetrics = metricsMap.get("pulsar_subscription_delayed_message_index_size_bytes"); + assertFalse(metrics.isEmpty()); + if (exposeTopicLevelMetrics) { + assertFalse(subMetrics.isEmpty()); + } else { + assertTrue(subMetrics.isEmpty()); + } int topicLevelNum = 0; int namespaceLevelNum = 0; @@ -462,14 +469,20 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex if (exposeTopicLevelMetrics && metric.tags.get("topic").equals(topic)) { Assert.assertTrue(metric.value > 0); topicLevelNum++; - if ("test_sub".equals(metric.tags.get("subscription"))) { - subscriptionLevelNum++; - } } else if (!exposeTopicLevelMetrics && metric.tags.get("namespace").equals(namespace)) { Assert.assertTrue(metric.value > 0); namespaceLevelNum++; } } + if (exposeTopicLevelMetrics) { + for (Metric metric : subMetrics) { + if (metric.tags.get("topic").equals(topic) && + subName.equals(metric.tags.get("subscription"))) { + Assert.assertTrue(metric.value > 0); + subscriptionLevelNum++; + } + } + } if (exposeTopicLevelMetrics) { Assert.assertTrue(topicLevelNum > 0); From aff9ec872b3f23aef90015dd9b81a8dccd6e9a07 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 Jan 2025 04:38:22 -0800 Subject: [PATCH 253/327] [improve] Support overriding java.net.preferIPv4Stack with OPTS (#23846) --- bin/bookkeeper | 2 +- bin/function-localrunner | 2 +- bin/pulsar | 2 +- bin/pulsar-admin-common.cmd | 2 +- bin/pulsar-admin-common.sh | 2 +- bin/pulsar-perf | 2 +- src/pulsar-io-gen.sh | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index ac7b622ac2363..6be45bffdfa87 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -201,7 +201,7 @@ OPTS="$OPTS --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk OPTS="-cp $BOOKIE_CLASSPATH $OPTS" # Disable ipv6 as it can cause issues -OPTS="$OPTS -Djava.net.preferIPv4Stack=true" +OPTS="-Djava.net.preferIPv4Stack=true $OPTS" OPTS="$OPTS $BOOKIE_MEM $BOOKIE_GC $BOOKIE_GC_LOG $BOOKIE_EXTRA_OPTS" diff --git a/bin/function-localrunner b/bin/function-localrunner index a47f3efa48609..90971277906db 100755 --- a/bin/function-localrunner +++ b/bin/function-localrunner @@ -131,7 +131,7 @@ fi # Ensure we can read bigger content from ZK. (It might be # rarely needed when trying to list many z-nodes under a # directory) -OPTS="$OPTS -Djute.maxbuffer=10485760 -Djava.net.preferIPv4Stack=true" +OPTS="-Djava.net.preferIPv4Stack=true $OPTS -Djute.maxbuffer=10485760" OPTS="-cp $PULSAR_CLASSPATH $OPTS" diff --git a/bin/pulsar b/bin/pulsar index 0125dd81e8bb4..25ac90769fbe8 100755 --- a/bin/pulsar +++ b/bin/pulsar @@ -273,7 +273,7 @@ OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`" # Ensure we can read bigger content from ZK. (It might be # rarely needed when trying to list many z-nodes under a # directory) -OPTS="$OPTS -Djute.maxbuffer=10485760 -Djava.net.preferIPv4Stack=true" +OPTS="-Djava.net.preferIPv4Stack=true $OPTS -Djute.maxbuffer=10485760" # Enable TCP keepalive for all Zookeeper client connections OPTS="$OPTS -Dzookeeper.clientTcpKeepAlive=true" diff --git a/bin/pulsar-admin-common.cmd b/bin/pulsar-admin-common.cmd index c59f0e9b424d3..12d61b0cd6327 100644 --- a/bin/pulsar-admin-common.cmd +++ b/bin/pulsar-admin-common.cmd @@ -60,7 +60,7 @@ for %%a in ("%PULSAR_LOG_CONF%") do SET "PULSAR_LOG_CONF_BASENAME=%%~nxa" set "PULSAR_CLASSPATH=%PULSAR_CLASSPATH%;%PULSAR_LOG_CONF_DIR%" set "OPTS=%OPTS% -Dlog4j.configurationFile="%PULSAR_LOG_CONF_BASENAME%"" -set "OPTS=%OPTS% -Djava.net.preferIPv4Stack=true" +set "OPTS=-Djava.net.preferIPv4Stack=true %OPTS%" REM Allow Netty to use reflection access set "OPTS=%OPTS% -Dio.netty.tryReflectionSetAccessible=true" diff --git a/bin/pulsar-admin-common.sh b/bin/pulsar-admin-common.sh index 336ff43c1a861..d83408bdf8bf7 100755 --- a/bin/pulsar-admin-common.sh +++ b/bin/pulsar-admin-common.sh @@ -104,7 +104,7 @@ fi PULSAR_CLASSPATH="$PULSAR_JAR:$PULSAR_CLASSPATH:$PULSAR_EXTRA_CLASSPATH" PULSAR_CLASSPATH="`dirname $PULSAR_LOG_CONF`:$PULSAR_CLASSPATH" OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`" -OPTS="$OPTS -Djava.net.preferIPv4Stack=true" +OPTS="-Djava.net.preferIPv4Stack=true $OPTS" # Allow Netty to use reflection access OPTS="$OPTS -Dio.netty.tryReflectionSetAccessible=true" diff --git a/bin/pulsar-perf b/bin/pulsar-perf index 9108a42ef994f..933fa3e6574f7 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -101,7 +101,7 @@ fi PULSAR_CLASSPATH="$PULSAR_JAR:$PULSAR_CLASSPATH:$PULSAR_EXTRA_CLASSPATH" PULSAR_CLASSPATH="`dirname $PULSAR_LOG_CONF`:$PULSAR_CLASSPATH" -OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF` -Djava.net.preferIPv4Stack=true" +OPTS="-Djava.net.preferIPv4Stack=true $OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`" # Allow Netty to use reflection access OPTS="$OPTS -Dio.netty.tryReflectionSetAccessible=true" diff --git a/src/pulsar-io-gen.sh b/src/pulsar-io-gen.sh index a3083e8f2b549..9f5b1a6058c7b 100755 --- a/src/pulsar-io-gen.sh +++ b/src/pulsar-io-gen.sh @@ -108,7 +108,7 @@ fi PULSAR_CLASSPATH="$PULSAR_JAR:$PULSAR_HOME/pulsar-io/docs/target/pulsar-io-docs.jar:$PULSAR_CLASSPATH:$PULSAR_EXTRA_CLASSPATH" PULSAR_CLASSPATH="`dirname $PULSAR_LOG_CONF`:$PULSAR_CLASSPATH" -OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF` -Djava.net.preferIPv4Stack=true" +OPTS="-Djava.net.preferIPv4Stack=true $OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`" OPTS="-cp $PULSAR_CLASSPATH $OPTS" OPTS="$OPTS $PULSAR_EXTRA_OPTS" From 492a869b8dbdb65f394454754260548020ceac7e Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 14 Jan 2025 22:39:14 +0800 Subject: [PATCH 254/327] [fix][misc] Honor dynamic log levels in log4j2.yaml (#23847) Signed-off-by: Zixuan Liu --- bin/pulsar | 12 ++++++++---- bin/pulsar-admin-common.cmd | 12 ++++++++---- bin/pulsar-perf | 12 ++++++++---- docker/pulsar/Dockerfile | 2 -- 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/bin/pulsar b/bin/pulsar index 25ac90769fbe8..4ff959a332c9f 100755 --- a/bin/pulsar +++ b/bin/pulsar @@ -303,8 +303,6 @@ else fi PULSAR_LOG_APPENDER=${PULSAR_LOG_APPENDER:-"RoutingAppender"} -PULSAR_LOG_LEVEL=${PULSAR_LOG_LEVEL:-"info"} -PULSAR_LOG_ROOT_LEVEL=${PULSAR_LOG_ROOT_LEVEL:-"${PULSAR_LOG_LEVEL}"} PULSAR_ROUTING_APPENDER_DEFAULT=${PULSAR_ROUTING_APPENDER_DEFAULT:-"Console"} if [ ! -d "$PULSAR_LOG_DIR" ]; then mkdir -p "$PULSAR_LOG_DIR" @@ -314,8 +312,14 @@ PULSAR_LOG_IMMEDIATE_FLUSH="${PULSAR_LOG_IMMEDIATE_FLUSH:-"false"}" #Configure log configuration system properties OPTS="$OPTS -Dpulsar.log.appender=$PULSAR_LOG_APPENDER" OPTS="$OPTS -Dpulsar.log.dir=$PULSAR_LOG_DIR" -OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL" -OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_ROOT_LEVEL" +if [ -n "$PULSAR_LOG_LEVEL" ]; then + OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL" +fi +if [ -n "$PULSAR_LOG_ROOT_LEVEL" ]; then + OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_ROOT_LEVEL" +elif [ -n "$PULSAR_LOG_LEVEL" ]; then + OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_LEVEL" +fi OPTS="$OPTS -Dpulsar.log.immediateFlush=$PULSAR_LOG_IMMEDIATE_FLUSH" OPTS="$OPTS -Dpulsar.routing.appender.default=$PULSAR_ROUTING_APPENDER_DEFAULT" # Configure log4j2 to disable servlet webapp detection so that Garbage free logging can be used diff --git a/bin/pulsar-admin-common.cmd b/bin/pulsar-admin-common.cmd index 12d61b0cd6327..265af6bafa8ad 100644 --- a/bin/pulsar-admin-common.cmd +++ b/bin/pulsar-admin-common.cmd @@ -83,14 +83,18 @@ set "OPTS=%OPTS% %PULSAR_EXTRA_OPTS%" if "%PULSAR_LOG_DIR%" == "" set "PULSAR_LOG_DIR=%PULSAR_HOME%\logs" if "%PULSAR_LOG_APPENDER%" == "" set "PULSAR_LOG_APPENDER=RoutingAppender" -if "%PULSAR_LOG_LEVEL%" == "" set "PULSAR_LOG_LEVEL=info" -if "%PULSAR_LOG_ROOT_LEVEL%" == "" set "PULSAR_LOG_ROOT_LEVEL=%PULSAR_LOG_LEVEL%" if "%PULSAR_ROUTING_APPENDER_DEFAULT%" == "" set "PULSAR_ROUTING_APPENDER_DEFAULT=Console" if "%PULSAR_LOG_IMMEDIATE_FLUSH%" == "" set "PULSAR_LOG_IMMEDIATE_FLUSH=false" set "OPTS=%OPTS% -Dpulsar.log.appender=%PULSAR_LOG_APPENDER%" set "OPTS=%OPTS% -Dpulsar.log.dir=%PULSAR_LOG_DIR%" -set "OPTS=%OPTS% -Dpulsar.log.level=%PULSAR_LOG_LEVEL%" -set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_ROOT_LEVEL%" +if not "%PULSAR_LOG_LEVEL%" == "" set "OPTS=%OPTS% -Dpulsar.log.level=%PULSAR_LOG_LEVEL%" +if not "%PULSAR_LOG_ROOT_LEVEL%" == "" ( + set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_ROOT_LEVEL%" +) else ( + if not "%PULSAR_LOG_LEVEL%" == "" ( + set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_LEVEL%" + ) +) set "OPTS=%OPTS% -Dpulsar.log.immediateFlush=%PULSAR_LOG_IMMEDIATE_FLUSH%" set "OPTS=%OPTS% -Dpulsar.routing.appender.default=%PULSAR_ROUTING_APPENDER_DEFAULT%" \ No newline at end of file diff --git a/bin/pulsar-perf b/bin/pulsar-perf index 933fa3e6574f7..2e957e4a3e9dc 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -125,14 +125,18 @@ OPTS="$OPTS $PULSAR_EXTRA_OPTS" # log directory & file PULSAR_LOG_APPENDER=${PULSAR_LOG_APPENDER:-"Console"} PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-perftest.log"} -PULSAR_LOG_LEVEL=${PULSAR_LOG_LEVEL:-"info"} -PULSAR_LOG_ROOT_LEVEL=${PULSAR_LOG_ROOT_LEVEL:-"${PULSAR_LOG_LEVEL}"} PULSAR_LOG_IMMEDIATE_FLUSH="${PULSAR_LOG_IMMEDIATE_FLUSH:-"false"}" #Configure log configuration system properties OPTS="$OPTS -Dpulsar.log.appender=$PULSAR_LOG_APPENDER" -OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL" -OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_ROOT_LEVEL" +if [ -n "$PULSAR_LOG_LEVEL" ]; then + OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL" +fi +if [ -n "$PULSAR_LOG_ROOT_LEVEL" ]; then + OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_ROOT_LEVEL" +elif [ -n "$PULSAR_LOG_LEVEL" ]; then + OPTS="$OPTS -Dpulsar.log.root.level=$PULSAR_LOG_LEVEL" +fi OPTS="$OPTS -Dpulsar.log.immediateFlush=$PULSAR_LOG_IMMEDIATE_FLUSH" OPTS="$OPTS -Dpulsar.log.dir=$PULSAR_LOG_DIR" OPTS="$OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE" diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 6f1f41755c919..63fadb62e64ae 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -135,8 +135,6 @@ COPY --from=snappy-java /tmp/libsnappyjava.so /usr/lib/libsnappyjava.so # The default is /pulsat/bin and cannot be written. ENV PULSAR_PID_DIR=/pulsar/logs -ENV PULSAR_ROOT_LOGGER=INFO,CONSOLE - COPY --from=pulsar /pulsar /pulsar WORKDIR /pulsar From b3641f068a862a738aacad14b0534cf012e4e584 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 16 Jan 2025 16:40:31 +0800 Subject: [PATCH 255/327] [fix][test]Fix flaky test testTopicUnloadAfterSessionRebuild (#23852) --- .../broker/service/ZkSessionExpireTest.java | 21 ++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java index dd0d7c423b643..b05dae2c3c2d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java @@ -26,6 +26,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; @@ -39,7 +40,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "flaky") +@Test(groups = "broker") public class ZkSessionExpireTest extends NetworkErrorTestBase { private java.util.function.Consumer settings; @@ -94,7 +95,7 @@ public void testTopicUnloadAfterSessionRebuild(boolean enableSystemTopic, Class admin2.namespaces().unload(defaultNamespace); // Confirm all brokers registered. - Awaitility.await().untilAsserted(() -> { + Awaitility.await().atMost(Duration.ofSeconds(20)).untilAsserted(() -> { assertEquals(getAvailableBrokers(pulsar1).size(), 2); assertEquals(getAvailableBrokers(pulsar2).size(), 2); }); @@ -160,7 +161,21 @@ public void testTopicUnloadAfterSessionRebuild(boolean enableSystemTopic, Class // Verify: the topic on broker-2 is fine. Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> { CompletableFuture> future = pulsar1.getBrokerService().getTopic(topicName, false); - assertTrue(future == null || future.isCompletedExceptionally()); + log.info("broker 1 topics {}", pulsar1.getBrokerService().getTopics().keySet()); + log.info("broker 2 topics {}", pulsar2.getBrokerService().getTopics().keySet()); + log.info("broker 1 bundles {}", pulsar1.getNamespaceService().getOwnershipCache().getOwnedBundles() + .keySet().stream().map(k -> k.getNamespaceObject().toString() + "/" + k.getBundleRange()) + .filter(s -> s.contains(defaultNamespace)).collect(Collectors.toList())); + log.info("broker 2 bundles {}", pulsar2.getNamespaceService().getOwnershipCache().getOwnedBundles() + .keySet().stream().map(k -> k.getNamespaceObject().toString() + "/" + k.getBundleRange()) + .filter(s -> s.contains(defaultNamespace)).collect(Collectors.toList())); + log.info("future: {}, isDone: {}, isCompletedExceptionally: {}", + future, future == null ? "null" : future.isDone(), + future, future == null ? "null" : future.isCompletedExceptionally()); + assertTrue(future == null + || !pulsar1.getBrokerService().getTopics().containsKey(topicName) + || (future.isDone() && !future.isCompletedExceptionally() && future.get().isEmpty()) + || future.isCompletedExceptionally()); }); Topic broker2Topic3 = pulsar2.getBrokerService().getTopic(topicName, false).join().get(); assertNotNull(broker2Topic3); From b55cd3e754e3bf7d8975ba3a3663eb9e64e139d6 Mon Sep 17 00:00:00 2001 From: Clay Johnson Date: Thu, 16 Jan 2025 05:41:11 -0600 Subject: [PATCH 256/327] [improve][ci] Publish build scans to develocity.apache.org (#23851) --- .github/workflows/ci-maven-cache-update.yaml | 2 +- .../workflows/ci-owasp-dependency-check.yaml | 2 +- .github/workflows/pulsar-ci-flaky.yaml | 2 +- .github/workflows/pulsar-ci.yaml | 20 +++++++++---------- .mvn/develocity.xml | 10 +++++----- .mvn/extensions.xml | 4 ++-- 6 files changed, 20 insertions(+), 20 deletions(-) diff --git a/.github/workflows/ci-maven-cache-update.yaml b/.github/workflows/ci-maven-cache-update.yaml index a673a30843417..6b0310e487123 100644 --- a/.github/workflows/ci-maven-cache-update.yaml +++ b/.github/workflows/ci-maven-cache-update.yaml @@ -50,7 +50,7 @@ jobs: name: Update Maven dependency cache for ${{ matrix.name }} env: JOB_NAME: Update Maven dependency cache for ${{ matrix.name }} - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} runs-on: ${{ matrix.runs-on }} timeout-minutes: 45 diff --git a/.github/workflows/ci-owasp-dependency-check.yaml b/.github/workflows/ci-owasp-dependency-check.yaml index a1c6dd594d3a2..6e60065b07ba9 100644 --- a/.github/workflows/ci-owasp-dependency-check.yaml +++ b/.github/workflows/ci-owasp-dependency-check.yaml @@ -34,7 +34,7 @@ jobs: name: Check ${{ matrix.branch }} env: JOB_NAME: Check ${{ matrix.branch }} - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} runs-on: ubuntu-22.04 timeout-minutes: 75 strategy: diff --git a/.github/workflows/pulsar-ci-flaky.yaml b/.github/workflows/pulsar-ci-flaky.yaml index bfc5140943172..514334d123c44 100644 --- a/.github/workflows/pulsar-ci-flaky.yaml +++ b/.github/workflows/pulsar-ci-flaky.yaml @@ -148,7 +148,7 @@ jobs: env: JOB_NAME: Flaky tests suite COLLECT_COVERAGE: "${{ needs.preconditions.outputs.collect_coverage }}" - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} TRACE_TEST_RESOURCE_CLEANUP: ${{ github.event_name == 'workflow_dispatch' && github.event.inputs.trace_test_resource_cleanup || 'off' }} TRACE_TEST_RESOURCE_CLEANUP_DIR: ${{ github.workspace }}/target/trace-test-resource-cleanup diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 87d8cd7cf9a78..6c71efe7e8549 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -147,7 +147,7 @@ jobs: name: Build and License check env: JOB_NAME: Build and License check - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} runs-on: ubuntu-22.04 timeout-minutes: 60 @@ -224,7 +224,7 @@ jobs: env: JOB_NAME: CI - Unit - ${{ matrix.name }} COLLECT_COVERAGE: "${{ needs.preconditions.outputs.collect_coverage }}" - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} TRACE_TEST_RESOURCE_CLEANUP: ${{ github.event_name == 'workflow_dispatch' && github.event.inputs.trace_test_resource_cleanup || 'off' }} TRACE_TEST_RESOURCE_CLEANUP_DIR: ${{ github.workspace }}/target/trace-test-resource-cleanup @@ -472,7 +472,7 @@ jobs: - linux/amd64 - linux/arm64 env: - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} IMAGE_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} steps: @@ -550,7 +550,7 @@ jobs: env: JOB_NAME: CI - Integration - ${{ matrix.name }} PULSAR_TEST_IMAGE_NAME: apachepulsar/java-test-image:latest - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} strategy: fail-fast: false @@ -831,7 +831,7 @@ jobs: needs: ['preconditions', 'build-and-license-check'] if: ${{ needs.preconditions.outputs.docs_only != 'true' }} env: - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} IMAGE_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} steps: @@ -957,7 +957,7 @@ jobs: env: JOB_NAME: CI - System - ${{ matrix.name }} PULSAR_TEST_IMAGE_NAME: apachepulsar/pulsar-test-latest-version:latest - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} strategy: fail-fast: false @@ -1187,7 +1187,7 @@ jobs: env: JOB_NAME: CI Flaky - System - ${{ matrix.name }} PULSAR_TEST_IMAGE_NAME: apachepulsar/pulsar-test-latest-version:latest - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} strategy: fail-fast: false @@ -1330,7 +1330,7 @@ jobs: needs: ['preconditions', 'integration-tests'] if: ${{ needs.preconditions.outputs.docs_only != 'true' }} env: - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} steps: - name: checkout @@ -1370,7 +1370,7 @@ jobs: contents: read security-events: write env: - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} CODEQL_LANGUAGE: java-kotlin steps: @@ -1431,7 +1431,7 @@ jobs: needs: [ 'preconditions', 'integration-tests' ] if: ${{ needs.preconditions.outputs.need_owasp == 'true' }} env: - DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} NIST_NVD_API_KEY: ${{ secrets.NIST_NVD_API_KEY }} steps: diff --git a/.mvn/develocity.xml b/.mvn/develocity.xml index 5c0fbb47c7217..399b37899985d 100644 --- a/.mvn/develocity.xml +++ b/.mvn/develocity.xml @@ -24,16 +24,16 @@ #{(env['GRADLE_ENTERPRISE_ACCESS_KEY']?.trim() > '' or env['DEVELOCITY_ACCESS_KEY']?.trim() > '') and !(env['GITHUB_HEAD_REF']?.matches('(?i).*(experiment|wip|private).*') or env['GITHUB_REPOSITORY']?.matches('(?i).*(experiment|wip|private).*'))} + pulsar - https://ge.apache.org + https://develocity.apache.org false - - true - true - #{isFalse(env['GITHUB_ACTIONS'])} + + authenticated + #{{'0.0.0.0'}} diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index eb998dc3471b8..8ceede33b9cdc 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -24,11 +24,11 @@ com.gradle develocity-maven-extension - 1.21.6 + 1.22.2 com.gradle common-custom-user-data-maven-extension - 2.0 + 2.0.1 From 8f04945c475dfc7d36d75878660b80015c97b176 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 16 Jan 2025 03:53:56 -0800 Subject: [PATCH 257/327] [fix][broker] Fix deadlock in Key_Shared PIP-379 implementation (#23854) --- .../broker/service/DrainingHashesTracker.java | 289 ++++++++++++------ .../apache/pulsar/broker/BrokerTestUtil.java | 17 +- .../client/api/KeySharedSubscriptionTest.java | 171 +++++++++++ 3 files changed, 389 insertions(+), 88 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java index 46762c844db6c..9bc5c5f1e44ec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java @@ -26,6 +26,8 @@ import java.util.Map; import java.util.PrimitiveIterator; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.locks.ReentrantReadWriteLock; import lombok.ToString; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.policies.data.DrainingHash; @@ -35,6 +37,11 @@ /** * A thread-safe map to store draining hashes in the consumer. + * The implementation uses read-write locks for ensuring thread-safe access. The high-level strategy to prevent + * deadlocks is to perform side-effects (calls to other collaborators which could have other exclusive locks) + * outside of the write lock. Early versions of this class had a problem where deadlocks could occur when + * a consumer operations happened at the same time as another thread requested topic stats which include + * the draining hashes state. This problem is avoided with the current implementation. */ @Slf4j public class DrainingHashesTracker { @@ -42,6 +49,7 @@ public class DrainingHashesTracker { private final UnblockingHandler unblockingHandler; // optimize the memory consumption of the map by using primitive int keys private final Int2ObjectOpenHashMap drainingHashes = new Int2ObjectOpenHashMap<>(); + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); int batchLevel; boolean unblockedWhileBatching; private final Map consumerDrainingHashesStatsMap = @@ -52,9 +60,14 @@ public class DrainingHashesTracker { */ @ToString public static class DrainingHashEntry { + private static final AtomicIntegerFieldUpdater REF_COUNT_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(DrainingHashEntry.class, "refCount"); + private static final AtomicIntegerFieldUpdater BLOCKED_COUNT_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(DrainingHashEntry.class, "blockedCount"); + private final Consumer consumer; - private int refCount; - private int blockedCount; + private volatile int refCount; + private volatile int blockedCount; /** * Constructs a new DrainingHashEntry with the specified Consumer. @@ -81,7 +94,7 @@ public Consumer getConsumer() { * Increments the reference count. */ void incrementRefCount() { - refCount++; + REF_COUNT_UPDATER.incrementAndGet(this); } /** @@ -90,14 +103,14 @@ void incrementRefCount() { * @return true if the reference count is zero, false otherwise */ boolean decrementRefCount() { - return --refCount == 0; + return REF_COUNT_UPDATER.decrementAndGet(this) == 0; } /** * Increments the blocked count. */ void incrementBlockedCount() { - blockedCount++; + BLOCKED_COUNT_UPDATER.incrementAndGet(this); } /** @@ -108,51 +121,89 @@ void incrementBlockedCount() { boolean isBlocking() { return blockedCount > 0; } + + /** + * Gets the current reference count. + * + * @return the current reference count + */ + int getRefCount() { + return refCount; + } + + /** + * Gets the current blocked count. + * + * @return the current blocked count + */ + int getBlockedCount() { + return blockedCount; + } } private class ConsumerDrainingHashesStats { private final RoaringBitmap drainingHashes = new RoaringBitmap(); - long drainingHashesClearedTotal; + private long drainingHashesClearedTotal; + private final ReentrantReadWriteLock statsLock = new ReentrantReadWriteLock(); - public synchronized void addHash(int stickyHash) { - drainingHashes.add(stickyHash); + public void addHash(int stickyHash) { + statsLock.writeLock().lock(); + try { + drainingHashes.add(stickyHash); + } finally { + statsLock.writeLock().unlock(); + } } - public synchronized boolean clearHash(int hash) { - drainingHashes.remove(hash); - drainingHashesClearedTotal++; - boolean empty = drainingHashes.isEmpty(); - if (log.isDebugEnabled()) { - log.debug("[{}] Cleared hash {} in stats. empty={} totalCleared={} hashes={}", - dispatcherName, hash, empty, drainingHashesClearedTotal, drainingHashes.getCardinality()); + public boolean clearHash(int hash) { + statsLock.writeLock().lock(); + try { + drainingHashes.remove(hash); + drainingHashesClearedTotal++; + boolean empty = drainingHashes.isEmpty(); + if (log.isDebugEnabled()) { + log.debug("[{}] Cleared hash {} in stats. empty={} totalCleared={} hashes={}", + dispatcherName, hash, empty, drainingHashesClearedTotal, drainingHashes.getCardinality()); + } + if (empty) { + // reduce memory usage by trimming the bitmap when the RoaringBitmap instance is empty + drainingHashes.trim(); + } + return empty; + } finally { + statsLock.writeLock().unlock(); } - return empty; } - public synchronized void updateConsumerStats(Consumer consumer, ConsumerStatsImpl consumerStats) { - int drainingHashesUnackedMessages = 0; - List drainingHashesStats = new ArrayList<>(); - PrimitiveIterator.OfInt hashIterator = drainingHashes.stream().iterator(); - while (hashIterator.hasNext()) { - int hash = hashIterator.nextInt(); - DrainingHashEntry entry = getEntry(hash); - if (entry == null) { - log.warn("[{}] Draining hash {} not found in the tracker for consumer {}", dispatcherName, hash, - consumer); - continue; + public void updateConsumerStats(Consumer consumer, ConsumerStatsImpl consumerStats) { + statsLock.readLock().lock(); + try { + int drainingHashesUnackedMessages = 0; + List drainingHashesStats = new ArrayList<>(); + PrimitiveIterator.OfInt hashIterator = drainingHashes.stream().iterator(); + while (hashIterator.hasNext()) { + int hash = hashIterator.nextInt(); + DrainingHashEntry entry = getEntry(hash); + if (entry == null) { + log.warn("[{}] Draining hash {} not found in the tracker for consumer {}", dispatcherName, hash, + consumer); + continue; + } + int unackedMessages = entry.getRefCount(); + DrainingHashImpl drainingHash = new DrainingHashImpl(); + drainingHash.hash = hash; + drainingHash.unackMsgs = unackedMessages; + drainingHash.blockedAttempts = entry.getBlockedCount(); + drainingHashesStats.add(drainingHash); + drainingHashesUnackedMessages += unackedMessages; } - int unackedMessages = entry.refCount; - DrainingHashImpl drainingHash = new DrainingHashImpl(); - drainingHash.hash = hash; - drainingHash.unackMsgs = unackedMessages; - drainingHash.blockedAttempts = entry.blockedCount; - drainingHashesStats.add(drainingHash); - drainingHashesUnackedMessages += unackedMessages; + consumerStats.drainingHashesCount = drainingHashesStats.size(); + consumerStats.drainingHashesClearedTotal = drainingHashesClearedTotal; + consumerStats.drainingHashesUnackedMessages = drainingHashesUnackedMessages; + consumerStats.drainingHashes = drainingHashesStats; + } finally { + statsLock.readLock().unlock(); } - consumerStats.drainingHashesCount = drainingHashesStats.size(); - consumerStats.drainingHashesClearedTotal = drainingHashesClearedTotal; - consumerStats.drainingHashesUnackedMessages = drainingHashesUnackedMessages; - consumerStats.drainingHashes = drainingHashesStats; } } @@ -179,49 +230,79 @@ public DrainingHashesTracker(String dispatcherName, UnblockingHandler unblocking * @param consumer the consumer * @param stickyHash the sticky hash */ - public synchronized void addEntry(Consumer consumer, int stickyHash) { + public void addEntry(Consumer consumer, int stickyHash) { if (stickyHash == 0) { throw new IllegalArgumentException("Sticky hash cannot be 0"); } - DrainingHashEntry entry = drainingHashes.get(stickyHash); - if (entry == null) { - if (log.isDebugEnabled()) { - log.debug("[{}] Adding and incrementing draining hash {} for consumer id:{} name:{}", dispatcherName, - stickyHash, consumer.consumerId(), consumer.consumerName()); - } - entry = new DrainingHashEntry(consumer); - drainingHashes.put(stickyHash, entry); - // update the consumer specific stats - consumerDrainingHashesStatsMap.computeIfAbsent(new ConsumerIdentityWrapper(consumer), - k -> new ConsumerDrainingHashesStats()).addHash(stickyHash); - } else if (entry.getConsumer() != consumer) { - throw new IllegalStateException( - "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash - + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer - + "."); - } else { - if (log.isDebugEnabled()) { - log.debug("[{}] Draining hash {} incrementing {} consumer id:{} name:{}", dispatcherName, stickyHash, - entry.refCount + 1, consumer.consumerId(), consumer.consumerName()); + + DrainingHashEntry entry; + ConsumerDrainingHashesStats addedStatsForNewEntry = null; + lock.writeLock().lock(); + try { + entry = drainingHashes.get(stickyHash); + if (entry == null) { + if (log.isDebugEnabled()) { + log.debug("[{}] Adding and incrementing draining hash {} for consumer id:{} name:{}", + dispatcherName, stickyHash, consumer.consumerId(), consumer.consumerName()); + } + entry = new DrainingHashEntry(consumer); + drainingHashes.put(stickyHash, entry); + // add the consumer specific stats + addedStatsForNewEntry = consumerDrainingHashesStatsMap + .computeIfAbsent(new ConsumerIdentityWrapper(consumer), k -> new ConsumerDrainingHashesStats()); + } else if (entry.getConsumer() != consumer) { + throw new IllegalStateException( + "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash + + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer + + "."); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Draining hash {} incrementing {} consumer id:{} name:{}", dispatcherName, + stickyHash, entry.getRefCount() + 1, consumer.consumerId(), consumer.consumerName()); + } } + } finally { + lock.writeLock().unlock(); } + // increment the reference count of the entry (applies to both new and existing entries) entry.incrementRefCount(); + + // perform side-effects outside of the lock to reduce chances for deadlocks + if (addedStatsForNewEntry != null) { + // add hash to added stats + addedStatsForNewEntry.addHash(stickyHash); + } } /** * Start a batch operation. There could be multiple nested batch operations. * The unblocking of sticky key hashes will be done only when the last batch operation ends. */ - public synchronized void startBatch() { - batchLevel++; + public void startBatch() { + lock.writeLock().lock(); + try { + batchLevel++; + } finally { + lock.writeLock().unlock(); + } } /** * End a batch operation. */ - public synchronized void endBatch() { - if (--batchLevel == 0 && unblockedWhileBatching) { - unblockedWhileBatching = false; + public void endBatch() { + boolean notifyUnblocking = false; + lock.writeLock().lock(); + try { + if (--batchLevel == 0 && unblockedWhileBatching) { + unblockedWhileBatching = false; + notifyUnblocking = true; + } + } finally { + lock.writeLock().unlock(); + } + // notify unblocking of the hash outside the lock + if (notifyUnblocking) { unblockingHandler.stickyKeyHashUnblocked(-1); } } @@ -231,13 +312,14 @@ public synchronized void endBatch() { * * @param consumer the consumer * @param stickyHash the sticky hash - * @param closing + * @param closing whether the consumer is closing */ - public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boolean closing) { + public void reduceRefCount(Consumer consumer, int stickyHash, boolean closing) { if (stickyHash == 0) { return; } - DrainingHashEntry entry = drainingHashes.get(stickyHash); + + DrainingHashEntry entry = getEntry(stickyHash); if (entry == null) { return; } @@ -252,24 +334,40 @@ public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boole log.debug("[{}] Draining hash {} removing consumer id:{} name:{}", dispatcherName, stickyHash, consumer.consumerId(), consumer.consumerName()); } - DrainingHashEntry removed = drainingHashes.remove(stickyHash); + + DrainingHashEntry removed; + boolean notifyUnblocking = false; + lock.writeLock().lock(); + try { + removed = drainingHashes.remove(stickyHash); + if (!closing && removed.isBlocking()) { + if (batchLevel > 0) { + unblockedWhileBatching = true; + } else { + notifyUnblocking = true; + } + } + } finally { + lock.writeLock().unlock(); + } + + // perform side-effects outside of the lock to reduce chances for deadlocks + // update the consumer specific stats ConsumerDrainingHashesStats drainingHashesStats = consumerDrainingHashesStatsMap.get(new ConsumerIdentityWrapper(consumer)); if (drainingHashesStats != null) { drainingHashesStats.clearHash(stickyHash); } - if (!closing && removed.isBlocking()) { - if (batchLevel > 0) { - unblockedWhileBatching = true; - } else { - unblockingHandler.stickyKeyHashUnblocked(stickyHash); - } + + // notify unblocking of the hash outside the lock + if (notifyUnblocking) { + unblockingHandler.stickyKeyHashUnblocked(stickyHash); } } else { if (log.isDebugEnabled()) { - log.debug("[{}] Draining hash {} decrementing {} consumer id:{} name:{}", dispatcherName, stickyHash, - entry.refCount, consumer.consumerId(), consumer.consumerName()); + log.debug("[{}] Draining hash {} decrementing {} consumer id:{} name:{}", dispatcherName, + stickyHash, entry.getRefCount(), consumer.consumerId(), consumer.consumerName()); } } } @@ -281,12 +379,12 @@ public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boole * @param stickyKeyHash the sticky key hash * @return true if the sticky key hash should be blocked, false otherwise */ - public synchronized boolean shouldBlockStickyKeyHash(Consumer consumer, int stickyKeyHash) { + public boolean shouldBlockStickyKeyHash(Consumer consumer, int stickyKeyHash) { if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { log.warn("[{}] Sticky key hash is not set. Allowing dispatching", dispatcherName); return false; } - DrainingHashEntry entry = drainingHashes.get(stickyKeyHash); + DrainingHashEntry entry = getEntry(stickyKeyHash); // if the entry is not found, the hash is not draining. Don't block the hash. if (entry == null) { return false; @@ -294,10 +392,14 @@ public synchronized boolean shouldBlockStickyKeyHash(Consumer consumer, int stic // hash has been reassigned to the original consumer, remove the entry // and don't block the hash if (entry.getConsumer() == consumer) { - log.info("[{}] Hash {} has been reassigned consumer {}. " - + "The draining hash entry with refCount={} will be removed.", - dispatcherName, stickyKeyHash, entry.getConsumer(), entry.refCount); - drainingHashes.remove(stickyKeyHash, entry); + log.info("[{}] Hash {} has been reassigned consumer {}. The draining hash entry with refCount={} will " + + "be removed.", dispatcherName, stickyKeyHash, entry.getConsumer(), entry.getRefCount()); + lock.writeLock().lock(); + try { + drainingHashes.remove(stickyKeyHash, entry); + } finally { + lock.writeLock().unlock(); + } return false; } // increment the blocked count which is used to determine if the hash is blocking @@ -313,16 +415,29 @@ public synchronized boolean shouldBlockStickyKeyHash(Consumer consumer, int stic * @param stickyKeyHash the sticky key hash * @return the draining hash entry, or null if not found */ - public synchronized DrainingHashEntry getEntry(int stickyKeyHash) { - return stickyKeyHash != 0 ? drainingHashes.get(stickyKeyHash) : null; + public DrainingHashEntry getEntry(int stickyKeyHash) { + if (stickyKeyHash == 0) { + return null; + } + lock.readLock().lock(); + try { + return drainingHashes.get(stickyKeyHash); + } finally { + lock.readLock().unlock(); + } } /** * Clear all entries in the draining hashes tracker. */ - public synchronized void clear() { - drainingHashes.clear(); - consumerDrainingHashesStatsMap.clear(); + public void clear() { + lock.writeLock().lock(); + try { + drainingHashes.clear(); + consumerDrainingHashesStatsMap.clear(); + } finally { + lock.writeLock().unlock(); + } } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index 6a41e86f8934e..8364cae53b223 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -224,7 +224,22 @@ public static String getJsonResourceAsString(String uri) { public static void receiveMessages(BiFunction, Message, Boolean> messageHandler, Duration quietTimeout, Consumer... consumers) { - FutureUtil.waitForAll(Arrays.stream(consumers) + receiveMessages(messageHandler, quietTimeout, Arrays.stream(consumers)); + } + + /** + * Receive messages concurrently from multiple consumers and handles them using the provided message handler. + * The message handler should return true if it wants to continue receiving more messages, false otherwise. + * + * @param messageHandler the message handler + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param consumers the consumers to receive messages from + * @param the message value type + */ + public static void receiveMessages(BiFunction, Message, Boolean> messageHandler, + Duration quietTimeout, + Stream> consumers) { + FutureUtil.waitForAll(consumers .map(consumer -> receiveMessagesAsync(consumer, quietTimeout, messageHandler)).toList()).join(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 92257c1df53f8..b7f11b3764150 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -55,6 +55,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -2359,4 +2360,174 @@ public void testOrderingAfterReconnects(KeySharedImplementationType impl) throws logTopicStats(topic); } } + + @Test(dataProvider = "currentImplementationType") + public void testDeliveryOfRemainingMessagesWithoutDeadlock(KeySharedImplementationType impl) throws Exception { + // don't set the unblock stuck subscription flag which is set to false by default, but for this test class + // it is enabled in the setup method + conf.setUnblockStuckSubscriptionEnabled(false); + + // this was the way how reproduce the deadlock issue https://github.com/apache/pulsar/issues/23848 + @Cleanup("interrupt") + Thread updateRatesThread = new Thread(() -> { + int count = 0; + // the deadlock issue typically reproduced before 100000 iterations + while (!Thread.currentThread().isInterrupted() && count++ < 200_000) { + pulsar.getBrokerService().updateRates(); + Thread.yield(); + if (count % 10000 == 0) { + log.info("updateRatesThread count: {}", count); + } + } + }, "update-rates-thread"); + updateRatesThread.start(); + + String topic = newUniqueName("testDeliveryOfRemainingMessagesWithoutDeadlock"); + int numberOfKeys = 100; + long pauseTime = 100L; + + @Cleanup + Producer producer = createProducer(topic, false); + + // create a consumer and close it to create a subscription + pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe() + .close(); + + Set remainingMessageValues = Collections.synchronizedSet(new HashSet<>()); + List, Message>> unackedMessages = new ArrayList<>(); + AtomicBoolean c2MessagesShouldBeUnacked = new AtomicBoolean(true); + Set keysForC2 = new HashSet<>(); + + Map> keyPositions = new HashMap<>(); + BiFunction, Message, Boolean> messageHandler = (consumer, msg) -> { + synchronized (this) { + String key = msg.getKey(); + if (c2MessagesShouldBeUnacked.get() && keysForC2.contains(key)) { + unackedMessages.add(Pair.of(consumer, msg)); + return true; + } + consumer.acknowledgeAsync(msg); + MessageIdAdv msgId = (MessageIdAdv) msg.getMessageId(); + Position currentPosition = PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId()); + Pair prevPair = keyPositions.get(key); + if (prevPair != null && prevPair.getLeft().compareTo(currentPosition) > 0) { + log.error("key: {} value: {} prev: {}/{} current: {}/{}", key, msg.getValue(), prevPair.getLeft(), + prevPair.getRight(), currentPosition, consumer.getConsumerName()); + fail("out of order"); + } + keyPositions.put(key, Pair.of(currentPosition, consumer.getConsumerName())); + boolean removed = remainingMessageValues.remove(msg.getValue()); + if (!removed) { + // duplicates are possible during reconnects, this is not an error + log.warn("Duplicate message: {} value: {}", msg.getMessageId(), msg.getValue()); + } + return true; + } + }; + + // Adding a new consumer. + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + + @Cleanup + Consumer c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + + @Cleanup + Consumer c3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + + // find keys that will be assigned to c2 + for (int i = 0; i < numberOfKeys; i++) { + String key = String.valueOf(i); + byte[] keyBytes = key.getBytes(UTF_8); + int hash = selector.makeStickyKeyHash(keyBytes); + if (selector.select(hash).consumerName().equals("c2")) { + keysForC2.add(key); + } + } + + // close c2 + c2.close(); + Thread.sleep(pauseTime); + + // produce messages with random keys + for (int i = 0; i < 1000; i++) { + String key = String.valueOf(random.nextInt(numberOfKeys)); + //log.info("Producing message with key: {} value: {}", key, i); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + } + + // consume the messages + receiveMessages(messageHandler, Duration.ofSeconds(1), c1, c3); + + c2MessagesShouldBeUnacked.set(false); + + // reconnect c2 + c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .startPaused(true) + .subscribe(); + + Thread.sleep(2 * pauseTime); + + // produce messages with c2 keys + List keysForC2List=new ArrayList<>(keysForC2); + for (int i = 1000; i < 1100; i++) { + String key = keysForC2List.get(random.nextInt(keysForC2List.size())); + //log.info("Producing message with key: {} value: {}", key, i); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + } + + Thread.sleep(2 * pauseTime); + + // ack the unacked messages to unblock c2 keys + unackedMessages.forEach(pair -> { + messageHandler.apply(pair.getLeft(), pair.getRight()); + }); + + Thread.sleep(50 * pauseTime); + + // resume c2 + c2.resume(); + + // consume the messages + receiveMessages(messageHandler, Duration.ofSeconds(2), c1, c2, c3); + + try { + assertEquals(remainingMessageValues, Collections.emptySet()); + } finally { + logTopicStats(topic); + } + } } From ea56ada4f3985c93b93c64d1361b3111cd98a37f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 16 Jan 2025 22:08:44 -0800 Subject: [PATCH 258/327] [fix][broker] Revert "[fix][broker] Cancel possible pending replay read in cancelPendingRead (#23384)" (#23855) --- .../broker/service/AbstractDispatcherMultipleConsumers.java | 4 ++++ .../persistent/PersistentDispatcherMultipleConsumers.java | 3 +-- .../PersistentDispatcherMultipleConsumersClassic.java | 3 +-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java index e3c2cf40cf318..bec02e94c79ab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java @@ -68,6 +68,10 @@ public SubType getType() { public abstract boolean isConsumerAvailable(Consumer consumer); + /** + * Cancel a possible pending read that is a Managed Cursor waiting to be notified for more entries. + * This won't cancel any other pending reads that are currently in progress. + */ protected void cancelPendingRead() {} /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index b1cd186c31784..fa03a260e131e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -687,9 +687,8 @@ public synchronized CompletableFuture disconnectAllConsumers( @Override protected void cancelPendingRead() { - if ((havePendingRead || havePendingReplayRead) && cursor.cancelPendingReadRequest()) { + if (havePendingRead && cursor.cancelPendingReadRequest()) { havePendingRead = false; - havePendingReplayRead = false; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java index 6ab7acfa56da8..910491e60b2cf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java @@ -600,9 +600,8 @@ public synchronized CompletableFuture disconnectAllConsumers( @Override protected void cancelPendingRead() { - if ((havePendingRead || havePendingReplayRead) && cursor.cancelPendingReadRequest()) { + if (havePendingRead && cursor.cancelPendingReadRequest()) { havePendingRead = false; - havePendingReplayRead = false; } } From 56adefaf3a244514a7eacbee17ac31b1a8739391 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 17 Jan 2025 14:48:46 +0800 Subject: [PATCH 259/327] [fix][client] Orphan producer when concurrently calling producer closing and reconnection (#23853) --- .../pulsar/client/impl/ProducerCloseTest.java | 41 ++++++ .../client/impl/ProducerReconnectionTest.java | 130 ++++++++++++++++++ .../pulsar/client/impl/ConnectionHandler.java | 7 +- .../pulsar/client/impl/ProducerImpl.java | 32 +++-- .../pulsar/client/impl/ProducerImplTest.java | 39 ------ 5 files changed, 197 insertions(+), 52 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerReconnectionTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerCloseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerCloseTest.java index 1141af88e72b0..623d8e7505e0d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerCloseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerCloseTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl; +import java.time.Duration; import lombok.Cleanup; import org.apache.commons.lang3.RandomStringUtils; import org.apache.pulsar.broker.service.Topic; @@ -71,6 +72,46 @@ public Object[][] produceConf() { }; } + /** + * Param1: Producer enableBatch or not + * Param2: Send in async way or not + */ + @DataProvider(name = "brokenPipeline") + public Object[][] brokenPipeline() { + return new Object[][]{ + {true}, + {false} + }; + } + + @Test(dataProvider = "brokenPipeline") + public void testProducerCloseCallback2(boolean brokenPipeline) throws Exception { + initClient(); + @Cleanup + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer() + .topic("testProducerClose") + .sendTimeout(5, TimeUnit.SECONDS) + .maxPendingMessages(0) + .enableBatching(false) + .create(); + final TypedMessageBuilder messageBuilder = producer.newMessage(); + final TypedMessageBuilder value = messageBuilder.value("test-msg".getBytes(StandardCharsets.UTF_8)); + producer.getClientCnx().channel().config().setAutoRead(false); + final CompletableFuture completableFuture = value.sendAsync(); + producer.closeAsync(); + Thread.sleep(3000); + if (brokenPipeline) { + //producer.getClientCnx().channel().config().setAutoRead(true); + producer.getClientCnx().channel().close(); + } else { + producer.getClientCnx().channel().config().setAutoRead(true); + } + Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> { + System.out.println(1); + Assert.assertTrue(completableFuture.isDone()); + }); + } + @Test(timeOut = 10_000) public void testProducerCloseCallback() throws Exception { initClient(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerReconnectionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerReconnectionTest.java new file mode 100644 index 0000000000000..e05e666d62efd --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerReconnectionTest.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.impl; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertTrue; +import java.nio.charset.StandardCharsets; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.ServerCnx; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class ProducerReconnectionTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testConcurrencyReconnectAndClose() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp_"); + admin.topics().createNonPartitionedTopic(topicName); + PulsarClientImpl client = (PulsarClientImpl) pulsarClient; + + // Create producer which will run with special steps. + ProducerBuilderImpl producerBuilder = (ProducerBuilderImpl) client.newProducer() + .blockIfQueueFull(false).maxPendingMessages(1).producerName("p1") + .enableBatching(true).topic(topicName); + CompletableFuture> producerFuture = new CompletableFuture<>(); + AtomicBoolean reconnectionStartTrigger = new AtomicBoolean(); + CountDownLatch reconnectingSignal = new CountDownLatch(1); + CountDownLatch closedSignal = new CountDownLatch(1); + ProducerImpl producer = new ProducerImpl<>(client, topicName, producerBuilder.getConf(), producerFuture, + -1, Schema.BYTES, null, Optional.empty()) { + @Override + ConnectionHandler initConnectionHandler() { + ConnectionHandler connectionHandler = super.initConnectionHandler(); + ConnectionHandler spyConnectionHandler = spy(connectionHandler); + doAnswer(invocation -> { + boolean result = (boolean) invocation.callRealMethod(); + if (reconnectionStartTrigger.get()) { + log.info("[testConcurrencyReconnectAndClose] verified state for reconnection"); + reconnectingSignal.countDown(); + closedSignal.await(); + log.info("[testConcurrencyReconnectAndClose] reconnected"); + } + return result; + }).when(spyConnectionHandler).isValidStateForReconnection(); + return spyConnectionHandler; + } + }; + log.info("[testConcurrencyReconnectAndClose] producer created"); + producerFuture.get(5, TimeUnit.SECONDS); + + // Reconnect. + log.info("[testConcurrencyReconnectAndClose] trigger a reconnection"); + ServerCnx serverCnx = (ServerCnx) pulsar.getBrokerService().getTopic(topicName, false).join() + .get().getProducers().values().iterator().next().getCnx(); + reconnectionStartTrigger.set(true); + serverCnx.ctx().close(); + producer.sendAsync("1".getBytes(StandardCharsets.UTF_8)); + Awaitility.await().untilAsserted(() -> { + assertNotEquals(producer.getPendingQueueSize(), 0); + }); + + // Close producer when reconnecting. + reconnectingSignal.await(); + log.info("[testConcurrencyReconnectAndClose] producer close"); + producer.closeAsync(); + Awaitility.await().untilAsserted(() -> { + HandlerState.State state1 = producer.getState(); + assertTrue(state1 == HandlerState.State.Closed || state1 == HandlerState.State.Closing); + }); + // give another thread time to call "signalToChangeStateToConnecting.await()". + closedSignal.countDown(); + + // Wait for reconnection. + Thread.sleep(3000); + + HandlerState.State state2 = producer.getState(); + log.info("producer state: {}", state2); + assertTrue(state2 == HandlerState.State.Closed || state2 == HandlerState.State.Closing); + assertEquals(producer.getPendingQueueSize(), 0); + + // Verify: ref is expected. + producer.close(); + admin.topics().delete(topicName); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java index 934985949197c..fb5b6788d08e5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl; +import com.google.common.annotations.VisibleForTesting; import java.net.InetSocketAddress; import java.net.URI; import java.util.Optional; @@ -192,13 +193,12 @@ public void connectionClosed(ClientCnx cnx, Optional initialConnectionDela duringConnect.set(false); state.client.getCnxPool().releaseConnection(cnx); if (CLIENT_CNX_UPDATER.compareAndSet(this, cnx, null)) { - if (!isValidStateForReconnection()) { + if (!state.changeToConnecting()) { log.info("[{}] [{}] Ignoring reconnection request (state: {})", state.topic, state.getHandlerName(), state.getState()); return; } long delayMs = initialConnectionDelayMs.orElse(backoff.next()); - state.setState(State.Connecting); log.info("[{}] [{}] Closed connection {} -- Will try again in {} s, hostUrl: {}", state.topic, state.getHandlerName(), cnx.channel(), delayMs / 1000.0, hostUrl.orElse(null)); state.client.timer().newTimeout(timeout -> { @@ -232,7 +232,8 @@ protected long switchClientCnx(ClientCnx clientCnx) { return EPOCH_UPDATER.incrementAndGet(this); } - private boolean isValidStateForReconnection() { + @VisibleForTesting + public boolean isValidStateForReconnection() { State state = this.state.getState(); switch (state) { case Uninitialized: diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 64b706cc5faa9..304b84961b7fa 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 @@ -303,16 +303,20 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration producersClosedCounter = ip.newCounter("pulsar.client.producer.closed", Unit.Sessions, "The number of producer sessions closed", topic, Attributes.empty()); - this.connectionHandler = new ConnectionHandler(this, + this.connectionHandler = initConnectionHandler(); + setChunkMaxMessageSize(); + grabCnx(); + producersOpenedCounter.increment(); + } + + ConnectionHandler initConnectionHandler() { + return new ConnectionHandler(this, new BackoffBuilder() .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), TimeUnit.NANOSECONDS) .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) .setMandatoryStop(Math.max(100, conf.getSendTimeoutMs() - 100), TimeUnit.MILLISECONDS) .create(), - this); - setChunkMaxMessageSize(); - grabCnx(); - producersOpenedCounter.increment(); + this); } private void setChunkMaxMessageSize() { @@ -1151,7 +1155,7 @@ public CompletableFuture handleOnce() { @Override - public CompletableFuture closeAsync() { + public synchronized CompletableFuture closeAsync() { final State currentState = getAndUpdateState(state -> { if (state == State.Closed) { return state; @@ -1179,11 +1183,11 @@ public CompletableFuture closeAsync() { CompletableFuture closeFuture = new CompletableFuture<>(); cnx.sendRequestWithId(cmd, requestId).handle((v, exception) -> { cnx.removeProducer(producerId); - closeAndClearPendingMessages(); if (exception == null || !cnx.ctx().channel().isActive()) { // Either we've received the success response for the close producer command from the broker, or the // connection did break in the meantime. In any case, the producer is gone. log.info("[{}] [{}] Closed Producer", topic, producerName); + closeAndClearPendingMessages(); closeFuture.complete(null); } else { closeFuture.completeExceptionally(exception); @@ -1795,6 +1799,12 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { // Because the state could have been updated while retrieving the connection, we set it back to connecting, // as long as the change from current state to connecting is a valid state change. if (!changeToConnecting()) { + if (getState() == State.Closing || getState() == State.Closed) { + // Producer was closed while reconnecting, close the connection to make sure the broker + // drops the producer on its side + failPendingMessages(cnx, + new PulsarClientException.ProducerFencedException("producer has been closed")); + } return CompletableFuture.completedFuture(null); } // We set the cnx reference before registering the producer on the cnx, so if the cnx breaks before creating @@ -1855,6 +1865,8 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { // Producer was closed while reconnecting, close the connection to make sure the broker // drops the producer on its side cnx.removeProducer(producerId); + failPendingMessages(cnx, + new PulsarClientException.ProducerFencedException("producer has been closed")); cnx.channel().close(); future.complete(null); return; @@ -2025,7 +2037,7 @@ private void closeProducerTasks() { private void resendMessages(ClientCnx cnx, long expectedEpoch) { cnx.ctx().channel().eventLoop().execute(() -> { - synchronized (this) { + synchronized (ProducerImpl.this) { if (getState() == State.Closing || getState() == State.Closed) { // Producer was closed while reconnecting, close the connection to make sure the broker // drops the producer on its side @@ -2181,7 +2193,7 @@ public void run(Timeout timeout) throws Exception { * This fails and clears the pending messages with the given exception. This method should be called from within the * ProducerImpl object mutex. */ - private void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { + private synchronized void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { if (cnx == null) { final AtomicInteger releaseCount = new AtomicInteger(); final boolean batchMessagingEnabled = isBatchMessagingEnabled(); @@ -2333,7 +2345,7 @@ private void batchMessageAndSend(boolean shouldScheduleNextBatchFlush) { } } - protected void processOpSendMsg(OpSendMsg op) { + protected synchronized void processOpSendMsg(OpSendMsg op) { if (op == null) { return; } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java index 5f690ead6c592..ce5a94592fdfe 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java @@ -25,16 +25,8 @@ import static org.testng.Assert.*; import java.nio.ByteBuffer; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; - -import io.netty.util.HashedWheelTimer; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.conf.ClientConfigurationData; -import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.client.impl.metrics.LatencyHistogram; -import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.mockito.Mockito; import org.testng.annotations.Test; @@ -74,35 +66,4 @@ public void testPopulateMessageSchema() { assertTrue(producer.populateMessageSchema(msg, null)); verify(msg).setSchemaState(MessageImpl.SchemaState.Ready); } - - @Test - public void testClearPendingMessageWhenCloseAsync() { - PulsarClientImpl client = mock(PulsarClientImpl.class); - Mockito.doReturn(1L).when(client).newProducerId(); - ClientConfigurationData clientConf = new ClientConfigurationData(); - clientConf.setStatsIntervalSeconds(-1); - Mockito.doReturn(clientConf).when(client).getConfiguration(); - Mockito.doReturn(new InstrumentProvider(null)).when(client).instrumentProvider(); - ConnectionPool connectionPool = mock(ConnectionPool.class); - Mockito.doReturn(1).when(connectionPool).genRandomKeyToSelectCon(); - Mockito.doReturn(connectionPool).when(client).getCnxPool(); - HashedWheelTimer timer = mock(HashedWheelTimer.class); - Mockito.doReturn(null).when(timer).newTimeout(Mockito.any(), Mockito.anyLong(), Mockito.any()); - Mockito.doReturn(timer).when(client).timer(); - ProducerConfigurationData producerConf = new ProducerConfigurationData(); - producerConf.setSendTimeoutMs(-1); - ProducerImpl producer = Mockito.spy(new ProducerImpl<>(client, "topicName", producerConf, null, 0, null, null, Optional.empty())); - - // make sure throw exception when send request to broker - ClientCnx clientCnx = mock(ClientCnx.class); - CompletableFuture tCompletableFuture = new CompletableFuture<>(); - tCompletableFuture.completeExceptionally(new PulsarClientException("error")); - when(clientCnx.sendRequestWithId(Mockito.any(), Mockito.anyLong())).thenReturn(tCompletableFuture); - Mockito.doReturn(clientCnx).when(producer).cnx(); - - // run closeAsync and verify - CompletableFuture voidCompletableFuture = producer.closeAsync(); - verify(producer).closeAndClearPendingMessages(); - } - } From 80157955b69d97fa0568fad4127e34e30e9a2b85 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 17 Jan 2025 15:57:24 +0800 Subject: [PATCH 260/327] [fix][test] Add reconsumeLater call in RetryTopicTest#testRetryTopicWithMultiTopic. (#23857) --- .../java/org/apache/pulsar/client/api/RetryTopicTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java index 91b97fa475817..d0e72deb87fc2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java @@ -464,7 +464,7 @@ public void testRetryTopicNameForCompatibility () throws Exception { } /** - * The test is disabled {@link https://github.com/apache/pulsar/issues/2647}. + * Test retry topic with multiple topics * @throws Exception */ @Test @@ -482,7 +482,6 @@ public void testRetryTopicWithMultiTopic() throws Exception { .subscriptionName("my-subscription") .subscriptionType(SubscriptionType.Shared) .enableRetry(true) - .ackTimeout(1, TimeUnit.SECONDS) .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build()) .receiverQueueSize(100) .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) @@ -518,6 +517,7 @@ public void testRetryTopicWithMultiTopic() throws Exception { Message message = consumer.receive(); log.info("consumer received message : {} {} - total = {}", message.getMessageId(), new String(message.getData()), ++totalReceived); + consumer.reconsumeLater(message, 1, TimeUnit.SECONDS); } while (totalReceived < sendMessages * (maxRedeliveryCount + 1)); int totalInDeadLetter = 0; From 4ee463342c0aedff5c0f543e0cc17455590f3632 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 20 Jan 2025 09:48:53 +0800 Subject: [PATCH 261/327] [fix] [ml] Fix cursor metadata compatability issue when switching the config unackedRangesOpenCacheSetEnabled (#23759) --- .../mledger/ManagedLedgerConfig.java | 7 + .../mledger/impl/ManagedCursorImpl.java | 48 ++++++- .../mledger/impl/RangeSetWrapper.java | 17 ++- .../mledger/impl/ManagedLedgerBkTest.java | 121 +++++++++++++++--- .../pulsar/broker/ServiceConfiguration.java | 4 + .../pulsar/broker/service/BrokerService.java | 2 + .../service/ManagedLedgerConfigTest.java | 70 ++++++++++ 7 files changed, 234 insertions(+), 35 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerConfigTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index a1e1deb503e20..6fc39170e851a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -93,6 +93,8 @@ public class ManagedLedgerConfig { @Getter @Setter private String shadowSourceName; + @Getter + private boolean persistIndividualAckAsLongArray; public boolean isCreateIfMissing() { return createIfMissing; @@ -103,6 +105,11 @@ public ManagedLedgerConfig setCreateIfMissing(boolean createIfMissing) { return this; } + public ManagedLedgerConfig setPersistIndividualAckAsLongArray(boolean persistIndividualAckAsLongArray) { + this.persistIndividualAckAsLongArray = persistIndividualAckAsLongArray; + return this; + } + /** * @return the lazyCursorRecovery */ diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 12cb22c161a90..203d48933f0a5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -180,7 +180,8 @@ public class ManagedCursorImpl implements ManagedCursor { protected volatile long messagesConsumedCounter; // Current ledger used to append the mark-delete position - private volatile LedgerHandle cursorLedger; + @VisibleForTesting + volatile LedgerHandle cursorLedger; // Wether the current cursorLedger is read-only or writable private boolean isCursorLedgerReadOnly = true; @@ -643,7 +644,22 @@ public void recoverIndividualDeletedMessages(PositionInfo positionInfo) { try { Map rangeMap = rangeList.stream().collect(Collectors.toMap(LongListMap::getKey, list -> list.getValuesList().stream().mapToLong(i -> i).toArray())); - individualDeletedMessages.build(rangeMap); + // Guarantee compatability for the config "unackedRangesOpenCacheSetEnabled". + if (getConfig().isUnackedRangesOpenCacheSetEnabled()) { + individualDeletedMessages.build(rangeMap); + } else { + RangeSetWrapper rangeSetWrapperV2 = new RangeSetWrapper<>(positionRangeConverter, + positionRangeReverseConverter, true, + getConfig().isPersistentUnackedRangesWithMultipleEntriesEnabled()); + rangeSetWrapperV2.build(rangeMap); + rangeSetWrapperV2.forEach(range -> { + individualDeletedMessages.addOpenClosed(range.lowerEndpoint().getLedgerId(), + range.lowerEndpoint().getEntryId(), range.upperEndpoint().getLedgerId(), + range.upperEndpoint().getEntryId()); + return true; + }); + rangeSetWrapperV2.clear(); + } } catch (Exception e) { log.warn("[{}]-{} Failed to recover individualDeletedMessages from serialized data", ledger.getName(), name, e); @@ -2376,7 +2392,14 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will // make the RangeSet recognize the "continuity" between adjacent Positions. - Position previousPosition = ledger.getPreviousPosition(position); + // Before https://github.com/apache/pulsar/pull/21105 is merged, the range does not support crossing + // multi ledgers, so the first position's entryId maybe "-1". + Position previousPosition; + if (position.getEntryId() == 0) { + previousPosition = PositionFactory.create(position.getLedgerId(), -1); + } else { + previousPosition = ledger.getPreviousPosition(position); + } individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this); @@ -3225,10 +3248,21 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin .addAllProperties(buildPropertiesMap(mdEntry.properties)); Map internalRanges = null; - try { - internalRanges = individualDeletedMessages.toRanges(getConfig().getMaxUnackedRangesToPersist()); - } catch (Exception e) { - log.warn("[{}]-{} Failed to serialize individualDeletedMessages", ledger.getName(), name, e); + /** + * Cursor will create the {@link #individualDeletedMessages} typed {@link LongPairRangeSet.DefaultRangeSet} if + * disabled the config {@link ManagedLedgerConfig#unackedRangesOpenCacheSetEnabled}. + * {@link LongPairRangeSet.DefaultRangeSet} never implemented the methods below: + * - {@link LongPairRangeSet#toRanges(int)}, which is used to serialize cursor metadata. + * - {@link LongPairRangeSet#build(Map)}, which is used to deserialize cursor metadata. + * Do not enable the feature that https://github.com/apache/pulsar/pull/9292 introduced, to avoid serialization + * and deserialization error. + */ + if (getConfig().isUnackedRangesOpenCacheSetEnabled() && getConfig().isPersistIndividualAckAsLongArray()) { + try { + internalRanges = individualDeletedMessages.toRanges(getConfig().getMaxUnackedRangesToPersist()); + } catch (Exception e) { + log.warn("[{}]-{} Failed to serialize individualDeletedMessages", ledger.getName(), name, e); + } } if (internalRanges != null && !internalRanges.isEmpty()) { piBuilder.addAllIndividualDeletedMessageRanges(buildLongPropertiesMap(internalRanges)); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index 11cce409bec54..76ac3e1be726c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -18,14 +18,12 @@ */ package org.apache.bookkeeper.mledger.impl; -import static java.util.Objects.requireNonNull; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Range; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; -import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; import org.roaringbitmap.RoaringBitSet; @@ -40,7 +38,6 @@ public class RangeSetWrapper> implements LongPairRangeSe private final LongPairRangeSet rangeSet; private final LongPairConsumer rangeConverter; - private final ManagedLedgerConfig config; private final boolean enableMultiEntry; /** @@ -53,13 +50,19 @@ public class RangeSetWrapper> implements LongPairRangeSe public RangeSetWrapper(LongPairConsumer rangeConverter, RangeBoundConsumer rangeBoundConsumer, ManagedCursorImpl managedCursor) { - requireNonNull(managedCursor); - this.config = managedCursor.getManagedLedger().getConfig(); + this(rangeConverter, rangeBoundConsumer, managedCursor.getConfig().isUnackedRangesOpenCacheSetEnabled(), + managedCursor.getConfig().isPersistentUnackedRangesWithMultipleEntriesEnabled()); + } + + public RangeSetWrapper(LongPairConsumer rangeConverter, + RangeBoundConsumer rangeBoundConsumer, + boolean unackedRangesOpenCacheSetEnabled, + boolean persistentUnackedRangesWithMultipleEntriesEnabled) { this.rangeConverter = rangeConverter; - this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() + this.rangeSet = unackedRangesOpenCacheSetEnabled ? new OpenLongPairRangeSet<>(rangeConverter, RoaringBitSet::new) : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter, rangeBoundConsumer); - this.enableMultiEntry = config.isPersistentUnackedRangesWithMultipleEntriesEnabled(); + this.enableMultiEntry = persistentUnackedRangesWithMultipleEntriesEnabled; } @Override diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java index 9635376a782d3..e23937afea2c9 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java @@ -20,13 +20,16 @@ import static org.apache.pulsar.common.util.PortManager.releaseLockedPort; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.fasterxml.jackson.databind.ObjectMapper; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; +import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.Future; @@ -34,8 +37,10 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperTestClient; +import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; @@ -49,18 +54,22 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.mledger.util.ThrowableToStringUtil; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.awaitility.Awaitility; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import io.netty.buffer.ByteBuf; import lombok.Cleanup; +@Slf4j public class ManagedLedgerBkTest extends BookKeeperClusterTestCase { + private final ObjectMapper jackson = new ObjectMapper(); + public ManagedLedgerBkTest() { super(2); } @@ -590,44 +599,114 @@ public void testPeriodicRollover() throws Exception { Awaitility.await().until(() -> cursorImpl.getCursorLedger() != currentLedgerId); } + @DataProvider(name = "unackedRangesOpenCacheSetEnabledPair") + public Object[][] unackedRangesOpenCacheSetEnabledPair() { + return new Object[][]{ + {false, true}, + {true, false}, + {true, true}, + {false, false} + }; + } + /** * This test validates that cursor serializes and deserializes individual-ack list from the bk-ledger. - * * @throws Exception */ - @Test - public void testUnackmessagesAndRecovery() throws Exception { + @Test(dataProvider = "unackedRangesOpenCacheSetEnabledPair") + public void testUnackmessagesAndRecoveryCompatibility(boolean enabled1, boolean enabled2) throws Exception { + final String mlName = "ml" + UUID.randomUUID().toString().replaceAll("-", ""); + final String cursorName = "c1"; ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig(); - factoryConf.setMaxCacheSize(0); - ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf); - - ManagedLedgerConfig config = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) + final ManagedLedgerConfig config1 = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) + .setAckQuorumSize(1).setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1) + .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1) + .setUnackedRangesOpenCacheSetEnabled(enabled1); + final ManagedLedgerConfig config2 = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) .setAckQuorumSize(1).setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1) - .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1); - ManagedLedger ledger = factory.open("my_test_unack_messages", config); - ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1) + .setUnackedRangesOpenCacheSetEnabled(enabled2); + + ManagedLedger ledger1 = factory.open(mlName, config1); + ManagedCursorImpl cursor1 = (ManagedCursorImpl) ledger1.openCursor(cursorName); int totalEntries = 100; for (int i = 0; i < totalEntries; i++) { - Position p = ledger.addEntry("entry".getBytes()); + Position p = ledger1.addEntry("entry".getBytes()); if (i % 2 == 0) { - cursor.delete(p); + cursor1.delete(p); } } + log.info("ack ranges: {}", cursor1.getIndividuallyDeletedMessagesSet().size()); - LongPairRangeSet unackMessagesBefore = cursor.getIndividuallyDeletedMessagesSet(); + // reopen and recover cursor + ledger1.close(); + ManagedLedger ledger2 = factory.open(mlName, config2); + ManagedCursorImpl cursor2 = (ManagedCursorImpl) ledger2.openCursor(cursorName); - ledger.close(); + log.info("before: {}", cursor1.getIndividuallyDeletedMessagesSet().asRanges()); + log.info("after : {}", cursor2.getIndividuallyDeletedMessagesSet().asRanges()); + assertEquals(cursor1.getIndividuallyDeletedMessagesSet().asRanges(), cursor2.getIndividuallyDeletedMessagesSet().asRanges()); + assertEquals(cursor1.markDeletePosition, cursor2.markDeletePosition); + + ledger2.close(); + factory.shutdown(); + } - // open and recover cursor - ledger = factory.open("my_test_unack_messages", config); - cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + @DataProvider(name = "booleans") + public Object[][] booleans() { + return new Object[][] { + {true}, + {false}, + }; + } - LongPairRangeSet unackMessagesAfter = cursor.getIndividuallyDeletedMessagesSet(); - assertTrue(unackMessagesBefore.equals(unackMessagesAfter)); + @Test(dataProvider = "booleans") + public void testConfigPersistIndividualAckAsLongArray(boolean enable) throws Exception { + final String mlName = "ml" + UUID.randomUUID().toString().replaceAll("-", ""); + final String cursorName = "c1"; + ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig(); + ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf); + final ManagedLedgerConfig config = new ManagedLedgerConfig() + .setEnsembleSize(1).setWriteQuorumSize(1).setAckQuorumSize(1) + .setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1).setMetadataAckQuorumSize(1) + .setMaxUnackedRangesToPersistInMetadataStore(1) + .setUnackedRangesOpenCacheSetEnabled(true).setPersistIndividualAckAsLongArray(enable); - ledger.close(); + ManagedLedger ledger1 = factory.open(mlName, config); + ManagedCursorImpl cursor1 = (ManagedCursorImpl) ledger1.openCursor(cursorName); + + // Write entries. + int totalEntries = 100; + List entries = new ArrayList<>(); + for (int i = 0; i < totalEntries; i++) { + Position p = ledger1.addEntry("entry".getBytes()); + entries.add(p); + } + // Make ack holes and trigger a mark deletion. + for (int i = totalEntries - 1; i >=0 ; i--) { + if (i % 2 == 0) { + cursor1.delete(entries.get(i)); + } + } + cursor1.markDelete(entries.get(9)); + Awaitility.await().untilAsserted(() -> { + assertEquals(cursor1.pendingMarkDeleteOps.size(), 0); + }); + + // Verify: the config affects. + long cursorLedgerLac = cursor1.cursorLedger.getLastAddConfirmed(); + LedgerEntry ledgerEntry = cursor1.cursorLedger.readEntries(cursorLedgerLac, cursorLedgerLac).nextElement(); + MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(ledgerEntry.getEntry()); + if (enable) { + assertNotEquals(positionInfo.getIndividualDeletedMessageRangesList().size(), 0); + } else { + assertEquals(positionInfo.getIndividualDeletedMessageRangesList().size(), 0); + } + + // cleanup + ledger1.close(); factory.shutdown(); } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index d27661d0ee65e..00e1113adfb9f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2266,6 +2266,10 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + " will only be tracked in memory and messages will be redelivered in case of" + " crashes.") private int managedLedgerMaxUnackedRangesToPersist = 10000; + @FieldContext( + category = CATEGORY_STORAGE_ML, + doc = "Whether persist cursor ack stats as long arrays, which will compress the data and reduce GC rate") + private boolean managedLedgerPersistIndividualAckAsLongArray = false; @FieldContext( category = CATEGORY_STORAGE_ML, doc = "If enabled, the maximum \"acknowledgment holes\" will not be limited and \"acknowledgment holes\" " diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index b8102488f493d..79e6fb2b02e31 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 @@ -2002,6 +2002,8 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T managedLedgerConfig .setMaxUnackedRangesToPersist(serviceConfig.getManagedLedgerMaxUnackedRangesToPersist()); + managedLedgerConfig + .setPersistIndividualAckAsLongArray(serviceConfig.isManagedLedgerPersistIndividualAckAsLongArray()); managedLedgerConfig.setPersistentUnackedRangesWithMultipleEntriesEnabled( serviceConfig.isPersistentUnackedRangesWithMultipleEntriesEnabled()); managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerConfigTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerConfigTest.java new file mode 100644 index 0000000000000..f9ce0d5019495 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerConfigTest.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.testng.Assert.assertEquals; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class ManagedLedgerConfigTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @DataProvider(name = "booleans") + public Object[][] booleans() { + return new Object[][] { + {true}, + {false}, + }; + } + + @Test(dataProvider = "booleans") + public void testConfigPersistIndividualAckAsLongArray(boolean enabled) throws Exception { + pulsar.getConfiguration().setManagedLedgerPersistIndividualAckAsLongArray(enabled); + final String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(tpName); + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, true).get().get(); + ManagedLedgerConfig mlConf = topic.getManagedLedger().getConfig(); + assertEquals(mlConf.isPersistIndividualAckAsLongArray(), enabled); + + // cleanup. + admin.topics().delete(tpName); + } +} + From a93e93dddddc3751a2d29727fc5226b689fed204 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 20 Jan 2025 22:08:08 +0800 Subject: [PATCH 262/327] [improve][broker] Remove spamming logs for customized managed ledger (#23862) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 08a481b7051c6..2325c8286a1be 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 @@ -3497,8 +3497,8 @@ public CompletableFuture updateOldPositionInfo() { TopicName topicName = TopicName.get(getName()); if (!(ledger.getCursors() instanceof ManagedCursorContainer managedCursorContainer)) { - return CompletableFuture.failedFuture(new IllegalStateException( - String.format("[%s] No valid cursors found. Skip update old position info.", topicName))); + // TODO: support this method with a customized managed ledger implementation + return CompletableFuture.completedFuture(null); } if (!hasBacklogs(brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck())) { From a96af729d805b679d0356b6dd2f6a1c66ad20252 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 20 Jan 2025 09:39:54 -0800 Subject: [PATCH 263/327] [improve] Upgrade to Netty 4.1.117.Final (#23863) --- buildtools/pom.xml | 2 +- .../server/src/assemble/LICENSE.bin.txt | 40 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 38 +++++++++--------- pom.xml | 2 +- 4 files changed, 41 insertions(+), 41 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 100ed1d973783..4ec8450b0c7ff 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -47,7 +47,7 @@ 4.1 10.14.2 3.1.2 - 4.1.116.Final + 4.1.117.Final 4.2.3 32.1.2-jre 1.10.12 diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 92b1e88f85280..5c9404167ae46 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -293,26 +293,26 @@ The Apache Software License, Version 2.0 - org.apache.commons-commons-lang3-3.11.jar - org.apache.commons-commons-text-1.10.0.jar * Netty - - io.netty-netty-buffer-4.1.116.Final.jar - - io.netty-netty-codec-4.1.116.Final.jar - - io.netty-netty-codec-dns-4.1.116.Final.jar - - io.netty-netty-codec-http-4.1.116.Final.jar - - io.netty-netty-codec-http2-4.1.116.Final.jar - - io.netty-netty-codec-socks-4.1.116.Final.jar - - io.netty-netty-codec-haproxy-4.1.116.Final.jar - - io.netty-netty-common-4.1.116.Final.jar - - io.netty-netty-handler-4.1.116.Final.jar - - io.netty-netty-handler-proxy-4.1.116.Final.jar - - io.netty-netty-resolver-4.1.116.Final.jar - - io.netty-netty-resolver-dns-4.1.116.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.116.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.116.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.116.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.116.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.116.Final.jar - - io.netty-netty-transport-native-epoll-4.1.116.Final-linux-aarch_64.jar - - io.netty-netty-transport-native-epoll-4.1.116.Final-linux-x86_64.jar - - io.netty-netty-transport-native-unix-common-4.1.116.Final.jar + - io.netty-netty-buffer-4.1.117.Final.jar + - io.netty-netty-codec-4.1.117.Final.jar + - io.netty-netty-codec-dns-4.1.117.Final.jar + - io.netty-netty-codec-http-4.1.117.Final.jar + - io.netty-netty-codec-http2-4.1.117.Final.jar + - io.netty-netty-codec-socks-4.1.117.Final.jar + - io.netty-netty-codec-haproxy-4.1.117.Final.jar + - io.netty-netty-common-4.1.117.Final.jar + - io.netty-netty-handler-4.1.117.Final.jar + - io.netty-netty-handler-proxy-4.1.117.Final.jar + - io.netty-netty-resolver-4.1.117.Final.jar + - io.netty-netty-resolver-dns-4.1.117.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.117.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.117.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.117.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.117.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.117.Final.jar + - io.netty-netty-transport-native-epoll-4.1.117.Final-linux-aarch_64.jar + - io.netty-netty-transport-native-epoll-4.1.117.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.117.Final.jar - io.netty-netty-tcnative-boringssl-static-2.0.69.Final.jar - io.netty-netty-tcnative-boringssl-static-2.0.69.Final-linux-aarch_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.69.Final-linux-x86_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 989a98470593d..fc97a98af6fde 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -347,22 +347,22 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.26.0.jar * Netty - - netty-buffer-4.1.116.Final.jar - - netty-codec-4.1.116.Final.jar - - netty-codec-dns-4.1.116.Final.jar - - netty-codec-http-4.1.116.Final.jar - - netty-codec-socks-4.1.116.Final.jar - - netty-codec-haproxy-4.1.116.Final.jar - - netty-common-4.1.116.Final.jar - - netty-handler-4.1.116.Final.jar - - netty-handler-proxy-4.1.116.Final.jar - - netty-resolver-4.1.116.Final.jar - - netty-resolver-dns-4.1.116.Final.jar - - netty-transport-4.1.116.Final.jar - - netty-transport-classes-epoll-4.1.116.Final.jar - - netty-transport-native-epoll-4.1.116.Final-linux-aarch_64.jar - - netty-transport-native-epoll-4.1.116.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.116.Final.jar + - netty-buffer-4.1.117.Final.jar + - netty-codec-4.1.117.Final.jar + - netty-codec-dns-4.1.117.Final.jar + - netty-codec-http-4.1.117.Final.jar + - netty-codec-socks-4.1.117.Final.jar + - netty-codec-haproxy-4.1.117.Final.jar + - netty-common-4.1.117.Final.jar + - netty-handler-4.1.117.Final.jar + - netty-handler-proxy-4.1.117.Final.jar + - netty-resolver-4.1.117.Final.jar + - netty-resolver-dns-4.1.117.Final.jar + - netty-transport-4.1.117.Final.jar + - netty-transport-classes-epoll-4.1.117.Final.jar + - netty-transport-native-epoll-4.1.117.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.117.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.117.Final.jar - netty-tcnative-boringssl-static-2.0.69.Final.jar - netty-tcnative-boringssl-static-2.0.69.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.69.Final-linux-x86_64.jar @@ -373,9 +373,9 @@ The Apache Software License, Version 2.0 - netty-incubator-transport-classes-io_uring-0.0.26.Final.jar - netty-incubator-transport-native-io_uring-0.0.26.Final-linux-aarch_64.jar - netty-incubator-transport-native-io_uring-0.0.26.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.116.Final.jar - - netty-resolver-dns-native-macos-4.1.116.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.116.Final-osx-x86_64.jar + - netty-resolver-dns-classes-macos-4.1.117.Final.jar + - netty-resolver-dns-native-macos-4.1.117.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.117.Final-osx-x86_64.jar * Prometheus client - simpleclient-0.16.0.jar - simpleclient_log4j2-0.16.0.jar diff --git a/pom.xml b/pom.xml index 047e014838f78..8cc8683ebf4f8 100644 --- a/pom.xml +++ b/pom.xml @@ -146,7 +146,7 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.7.1 - 4.1.116.Final + 4.1.117.Final 0.0.26.Final 9.4.56.v20240826 2.5.2 From 3c2ec2bf8bfd94eded46b42c5089dd8321afd096 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 20 Jan 2025 10:42:51 -0800 Subject: [PATCH 264/327] [improve][broker] Improve Consumer.equals performance (#23864) --- .../main/java/org/apache/pulsar/broker/service/Consumer.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 0baba13cc0b07..b46e10a20fd34 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 @@ -1048,6 +1048,9 @@ public CompletableFuture checkPermissionsAsync() { @Override public boolean equals(Object obj) { + if (this == obj) { + return true; + } if (obj instanceof Consumer) { Consumer other = (Consumer) obj; return consumerId == other.consumerId && Objects.equals(cnx.clientAddress(), other.cnx.clientAddress()); From e5bd77419e91d1602731cd0c1d02a738e1b7ebc7 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Tue, 21 Jan 2025 10:12:04 +0800 Subject: [PATCH 265/327] [feat][misc] upgrade oxia java client to 0.5.0 (#23871) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 5c9404167ae46..8e45de3a98da5 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -481,8 +481,8 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-api-0.4.10.jar - - io.streamnative.oxia-oxia-client-0.4.10.jar + - io.streamnative.oxia-oxia-client-api-0.5.0.jar + - io.streamnative.oxia-oxia-client-0.5.0.jar * OpenHFT - net.openhft-zero-allocation-hashing-0.16.jar * Java JSON WebTokens diff --git a/pom.xml b/pom.xml index 8cc8683ebf4f8..da3d8479b1fe4 100644 --- a/pom.xml +++ b/pom.xml @@ -250,7 +250,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.7 - 0.4.10 + 0.5.0 2.0 1.10.12 5.5.0 From 87fb442c223d47d8a426b44575981345d7a23481 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 22 Jan 2025 22:36:40 -0800 Subject: [PATCH 266/327] [improve][test] Add solution to PulsarMockBookKeeper for intercepting reads (#23875) --- .../client/PulsarMockBookKeeper.java | 8 +++- .../client/PulsarMockLedgerHandle.java | 2 +- .../client/PulsarMockReadHandle.java | 31 +++++++++----- .../PulsarMockReadHandleInterceptor.java | 40 +++++++++++++++++++ 4 files changed, 68 insertions(+), 13 deletions(-) create mode 100644 testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandleInterceptor.java diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java index 1e979206e16d3..344173c30918d 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -40,6 +40,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import lombok.Getter; +import lombok.Setter; import org.apache.bookkeeper.client.AsyncCallback.CreateCallback; import org.apache.bookkeeper.client.AsyncCallback.DeleteCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; @@ -96,6 +98,9 @@ public static Collection getMockEnsemble() { final Queue addEntryResponseDelaysMillis = new ConcurrentLinkedQueue<>(); final List> failures = new ArrayList<>(); final List> addEntryFailures = new ArrayList<>(); + @Setter + @Getter + private volatile PulsarMockReadHandleInterceptor readHandleInterceptor; public PulsarMockBookKeeper(OrderedExecutor orderedExecutor) throws Exception { this.orderedExecutor = orderedExecutor; @@ -250,7 +255,8 @@ public CompletableFuture execute() { return FutureUtils.exception(new BKException.BKUnauthorizedAccessException()); } else { return FutureUtils.value(new PulsarMockReadHandle(PulsarMockBookKeeper.this, ledgerId, - lh.getLedgerMetadata(), lh.entries)); + lh.getLedgerMetadata(), lh.entries, + PulsarMockBookKeeper.this::getReadHandleInterceptor)); } }); } diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java index aa61e541d0d6b..d30684e604670 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java @@ -73,7 +73,7 @@ public PulsarMockLedgerHandle(PulsarMockBookKeeper bk, long id, this.digest = digest; this.passwd = Arrays.copyOf(passwd, passwd.length); - readHandle = new PulsarMockReadHandle(bk, id, getLedgerMetadata(), entries); + readHandle = new PulsarMockReadHandle(bk, id, getLedgerMetadata(), entries, bk::getReadHandleInterceptor); } @Override diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandle.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandle.java index a4361f62254e4..9f3f4969199ce 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandle.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandle.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.LastConfirmedAndEntry; import org.apache.bookkeeper.client.api.LedgerEntries; @@ -40,28 +41,36 @@ class PulsarMockReadHandle implements ReadHandle { private final long ledgerId; private final LedgerMetadata metadata; private final List entries; + private final Supplier readHandleInterceptorSupplier; PulsarMockReadHandle(PulsarMockBookKeeper bk, long ledgerId, LedgerMetadata metadata, - List entries) { + List entries, + Supplier readHandleInterceptorSupplier) { this.bk = bk; this.ledgerId = ledgerId; this.metadata = metadata; this.entries = entries; + this.readHandleInterceptorSupplier = readHandleInterceptorSupplier; } @Override public CompletableFuture readAsync(long firstEntry, long lastEntry) { return bk.getProgrammedFailure().thenComposeAsync((res) -> { - log.debug("readEntries: first={} last={} total={}", firstEntry, lastEntry, entries.size()); - List seq = new ArrayList<>(); - long entryId = firstEntry; - while (entryId <= lastEntry && entryId < entries.size()) { - seq.add(entries.get((int) entryId++).duplicate()); - } - log.debug("Entries read: {}", seq); - - return FutureUtils.value(LedgerEntriesImpl.create(seq)); - }); + log.debug("readEntries: first={} last={} total={}", firstEntry, lastEntry, entries.size()); + List seq = new ArrayList<>(); + long entryId = firstEntry; + while (entryId <= lastEntry && entryId < entries.size()) { + seq.add(entries.get((int) entryId++).duplicate()); + } + log.debug("Entries read: {}", seq); + LedgerEntriesImpl ledgerEntries = LedgerEntriesImpl.create(seq); + PulsarMockReadHandleInterceptor pulsarMockReadHandleInterceptor = readHandleInterceptorSupplier.get(); + if (pulsarMockReadHandleInterceptor != null) { + return pulsarMockReadHandleInterceptor.interceptReadAsync(ledgerId, firstEntry, lastEntry, + ledgerEntries); + } + return FutureUtils.value(ledgerEntries); + }); } @Override diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandleInterceptor.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandleInterceptor.java new file mode 100644 index 0000000000000..acee87b0f77f4 --- /dev/null +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockReadHandleInterceptor.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.client; + +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.client.api.LedgerEntries; + +/** + * Interceptor interface for intercepting read handle readAsync operations. + * This is useful for testing purposes, for example for introducing delays. + */ +public interface PulsarMockReadHandleInterceptor { + /** + * Intercepts the readAsync operation on a read handle. + * + * @param ledgerId ledger id + * @param firstEntry first entry to read + * @param lastEntry last entry to read + * @param entries entries that would be returned by the read operation + * @return CompletableFuture that will complete with the entries to return + */ + CompletableFuture interceptReadAsync(long ledgerId, long firstEntry, long lastEntry, + LedgerEntries entries); +} From 52e8730613c36008ea57a0ca5c10231512232d7e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 22 Jan 2025 22:48:46 -0800 Subject: [PATCH 267/327] [fix][test] Fix quiet time implementation in BrokerTestUtil.receiveMessages (#23876) --- .../apache/pulsar/broker/BrokerTestUtil.java | 41 +++++-- .../pulsar/broker/BrokerTestUtilTest.java | 115 ++++++++++++++++++ 2 files changed, 143 insertions(+), 13 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtilTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index 8364cae53b223..e97928c4c66e0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -39,10 +39,12 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.stream.Stream; import lombok.SneakyThrows; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -239,33 +241,46 @@ public static void receiveMessages(BiFunction, Message, Boole public static void receiveMessages(BiFunction, Message, Boolean> messageHandler, Duration quietTimeout, Stream> consumers) { + long quietTimeoutNanos = quietTimeout.toNanos(); + AtomicLong lastMessageReceivedNanos = new AtomicLong(System.nanoTime()); FutureUtil.waitForAll(consumers - .map(consumer -> receiveMessagesAsync(consumer, quietTimeout, messageHandler)).toList()).join(); + .map(consumer -> receiveMessagesAsync(consumer, quietTimeoutNanos, quietTimeoutNanos, messageHandler, + lastMessageReceivedNanos)).toList()).join(); } // asynchronously receive messages from a consumer and handle them using the provided message handler // the benefit is that multiple consumers can be concurrently consumed without the need to have multiple threads // this is useful in tests where multiple consumers are needed to test the functionality - private static CompletableFuture receiveMessagesAsync(Consumer consumer, Duration quietTimeout, - BiFunction, Message, Boolean> - messageHandler) { - CompletableFuture> receiveFuture = consumer.receiveAsync(); - return receiveFuture - .orTimeout(quietTimeout.toMillis(), TimeUnit.MILLISECONDS) + private static CompletableFuture receiveMessagesAsync(Consumer consumer, + long quietTimeoutNanos, + long receiveTimeoutNanos, + BiFunction, Message, Boolean> + messageHandler, + AtomicLong lastMessageReceivedNanos) { + return consumer.receiveAsync() + .orTimeout(receiveTimeoutNanos, TimeUnit.NANOSECONDS) .handle((msg, t) -> { + long currentNanos = System.nanoTime(); if (t != null) { if (t instanceof TimeoutException) { - // cancel the receive future so that Pulsar client can clean up the resources - receiveFuture.cancel(false); - return false; + long sinceLastMessageReceivedNanos = currentNanos - lastMessageReceivedNanos.get(); + if (sinceLastMessageReceivedNanos > quietTimeoutNanos) { + return Pair.of(false, 0L); + } else { + return Pair.of(true, quietTimeoutNanos - sinceLastMessageReceivedNanos); + } } else { throw FutureUtil.wrapToCompletionException(t); } } - return messageHandler.apply(consumer, msg); - }).thenComposeAsync(receiveMore -> { + lastMessageReceivedNanos.set(currentNanos); + return Pair.of(messageHandler.apply(consumer, msg), quietTimeoutNanos); + }).thenComposeAsync(receiveMoreAndNextTimeout -> { + boolean receiveMore = receiveMoreAndNextTimeout.getLeft(); if (receiveMore) { - return receiveMessagesAsync(consumer, quietTimeout, messageHandler); + Long nextReceiveTimeoutNanos = receiveMoreAndNextTimeout.getRight(); + return receiveMessagesAsync(consumer, quietTimeoutNanos, nextReceiveTimeoutNanos, + messageHandler, lastMessageReceivedNanos); } else { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtilTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtilTest.java new file mode 100644 index 0000000000000..90b917a319c71 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtilTest.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.common.util.FutureUtil; +import org.testng.annotations.Test; + +@Slf4j +public class BrokerTestUtilTest { + @Test + public void testReceiveMessagesQuietTime() throws Exception { + // Mock consumers + Consumer consumer1 = mock(Consumer.class); + Consumer consumer2 = mock(Consumer.class); + + long consumer1DelayMs = 300L; + long consumer2DelayMs = 400L; + long quietTimeMs = 500L; + + // Define behavior for receiveAsync with delay + AtomicBoolean consumer1FutureContinueSupplying = new AtomicBoolean(true); + when(consumer1.receiveAsync()).thenAnswer(invocation -> { + if (consumer1FutureContinueSupplying.get()) { + CompletableFuture messageCompletableFuture = + CompletableFuture.supplyAsync(() -> mock(Message.class), + CompletableFuture.delayedExecutor(consumer1DelayMs, TimeUnit.MILLISECONDS)); + consumer1FutureContinueSupplying.set(false); + // continue supplying while the future is cancelled or timed out + FutureUtil.whenCancelledOrTimedOut(messageCompletableFuture, () -> { + consumer1FutureContinueSupplying.set(true); + }); + return messageCompletableFuture; + } else { + return new CompletableFuture<>(); + } + }); + AtomicBoolean consumer2FutureContinueSupplying = new AtomicBoolean(true); + when(consumer2.receiveAsync()).thenAnswer(invocation -> { + if (consumer2FutureContinueSupplying.get()) { + CompletableFuture messageCompletableFuture = + CompletableFuture.supplyAsync(() -> mock(Message.class), + CompletableFuture.delayedExecutor(consumer2DelayMs, TimeUnit.MILLISECONDS)); + consumer2FutureContinueSupplying.set(false); + // continue supplying while the future is cancelled or timed out + FutureUtil.whenCancelledOrTimedOut(messageCompletableFuture, () -> { + consumer2FutureContinueSupplying.set(true); + }); + return messageCompletableFuture; + } else { + return new CompletableFuture<>(); + } + }); + + // Atomic variables to track message handling + AtomicInteger messageCount = new AtomicInteger(0); + + // Message handler + BiFunction, Message, Boolean> messageHandler = (consumer, msg) -> { + messageCount.incrementAndGet(); + return true; + }; + + // Track start time + long startTime = System.nanoTime(); + + // Call receiveMessages method + BrokerTestUtil.receiveMessages(messageHandler, Duration.ofMillis(quietTimeMs), consumer1, consumer2); + + // Track end time + long endTime = System.nanoTime(); + + // Verify that messages were attempted to be received + verify(consumer1, times(3)).receiveAsync(); + verify(consumer2, times(2)).receiveAsync(); + + // Verify that the message handler was called + assertEquals(messageCount.get(), 2); + + // Verify the time spent is as expected (within a reasonable margin) + long durationMillis = TimeUnit.NANOSECONDS.toMillis(endTime - startTime); + assertThat(durationMillis).isBetween(consumer2DelayMs + quietTimeMs, + consumer2DelayMs + quietTimeMs + (quietTimeMs / 2)); + } +} \ No newline at end of file From eaf9ac11eafb5c07edb4ff1fa15d7d4f478e2011 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 23 Jan 2025 02:21:44 -0800 Subject: [PATCH 268/327] [fix][broker] PIP-379 Key_Shared implementation race condition causing out-of-order message delivery (#23874) --- ...PersistentDispatcherMultipleConsumers.java | 3 + ...tStickyKeyDispatcherMultipleConsumers.java | 57 ++- ...edSubscriptionDisabledBrokerCacheTest.java | 348 ++++++++++++++++++ pulsar-broker/src/test/resources/log4j2.xml | 8 +- 4 files changed, 399 insertions(+), 17 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionDisabledBrokerCacheTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index fa03a260e131e..82b96c365072f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -1422,6 +1422,9 @@ public void cursorIsReset() { protected boolean addMessageToReplay(long ledgerId, long entryId, long stickyKeyHash) { if (checkIfMessageIsUnacked(ledgerId, entryId)) { + if (log.isDebugEnabled()) { + log.debug("[{}] Adding message to replay for {}:{} hash: {}", name, ledgerId, entryId, stickyKeyHash); + } redeliveryMessages.add(ledgerId, entryId, stickyKeyHash); return true; } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 1a3e2f706cba8..8bddbde02c974 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -20,6 +20,8 @@ import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import com.google.common.annotations.VisibleForTesting; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; +import it.unimi.dsi.fastutil.ints.IntSet; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -407,6 +409,8 @@ private Map> filterAndGroupEntriesForDispatching(List blockedByHashConsumers = lookAheadAllowed && readType == ReadType.Normal ? new HashSet<>() : null; // in replay read mode, keep track of consumers for entries, used for look-ahead check Set consumersForEntriesForLookaheadCheck = lookAheadAllowed ? new HashSet<>() : null; + // track already blocked hashes to block any further messages with the same hash + IntSet alreadyBlockedHashes = new IntOpenHashSet(); for (Entry inputEntry : entries) { EntryAndMetadata entry; @@ -419,24 +423,29 @@ private Map> filterAndGroupEntriesForDispatching(List new MutableInt(getAvailablePermits(consumer))); - // a consumer was found for the sticky key hash and the entry can be dispatched - if (permits.intValue() > 0 - && canDispatchEntry(consumer, entry, readType, stickyKeyHash, blockedByHash)) { - // decrement the permits for the consumer - permits.decrement(); - // allow the entry to be dispatched - dispatchEntry = true; + // check if the hash is already blocked + boolean hashIsAlreadyBlocked = alreadyBlockedHashes.contains(stickyKeyHash); + if (!hashIsAlreadyBlocked) { + consumer = selector.select(stickyKeyHash); + if (consumer != null) { + if (lookAheadAllowed) { + consumersForEntriesForLookaheadCheck.add(consumer); + } + blockedByHash = lookAheadAllowed && readType == ReadType.Normal ? new MutableBoolean(false) : null; + MutableInt permits = + permitsForConsumer.computeIfAbsent(consumer, + k -> new MutableInt(getAvailablePermits(k))); + // a consumer was found for the sticky key hash and the entry can be dispatched + if (permits.intValue() > 0 + && canDispatchEntry(consumer, entry, readType, stickyKeyHash, blockedByHash)) { + // decrement the permits for the consumer + permits.decrement(); + // allow the entry to be dispatched + dispatchEntry = true; + } } } if (dispatchEntry) { @@ -445,6 +454,10 @@ && canDispatchEntry(consumer, entry, readType, stickyKeyHash, blockedByHash)) { entriesGroupedByConsumer.computeIfAbsent(consumer, k -> new ArrayList<>()); consumerEntries.add(entry); } else { + if (!hashIsAlreadyBlocked) { + // the hash is blocked, add it to the set of blocked hashes + alreadyBlockedHashes.add(stickyKeyHash); + } if (blockedByHash != null && blockedByHash.isTrue()) { // the entry is blocked by hash, add the consumer to the blocked set blockedByHashConsumers.add(consumer); @@ -536,6 +549,9 @@ private class ReplayPositionFilter implements Predicate { // tracks the available permits for each consumer for the duration of the filter usage // the filter is stateful and shouldn't be shared or reused later private final Map availablePermitsMap = new HashMap<>(); + // tracks the hashes that have been blocked during the filtering + // it is necessary to block all later messages after a hash gets blocked so that ordering is preserved + private final Set alreadyBlockedHashes = new HashSet<>(); @Override public boolean test(Position position) { @@ -553,25 +569,34 @@ public boolean test(Position position) { } return true; } + // check if the hash is already blocked, if so, then replaying of the position should be skipped + // to preserve ordering + if (alreadyBlockedHashes.contains(stickyKeyHash)) { + return false; + } // find the consumer for the sticky key hash Consumer consumer = selector.select(stickyKeyHash.intValue()); // skip replaying the message position if there's no assigned consumer if (consumer == null) { + alreadyBlockedHashes.add(stickyKeyHash); return false; } + // lookup the available permits for the consumer MutableInt availablePermits = availablePermitsMap.computeIfAbsent(consumer, k -> new MutableInt(getAvailablePermits(consumer))); // skip replaying the message position if the consumer has no available permits if (availablePermits.intValue() <= 0) { + alreadyBlockedHashes.add(stickyKeyHash); return false; } if (drainingHashesRequired && drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash.intValue())) { // the hash is draining and the consumer is not the draining consumer + alreadyBlockedHashes.add(stickyKeyHash); return false; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionDisabledBrokerCacheTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionDisabledBrokerCacheTest.java new file mode 100644 index 0000000000000..45f776b41e78a --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionDisabledBrokerCacheTest.java @@ -0,0 +1,348 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; +import static org.assertj.core.api.SoftAssertions.assertSoftly; +import static org.testng.Assert.fail; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import lombok.Cleanup; +import lombok.SneakyThrows; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.tests.KeySharedImplementationType; +import org.awaitility.Awaitility; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; +import org.testng.annotations.Test; + +@Test(groups = "broker-impl") +public class KeySharedSubscriptionDisabledBrokerCacheTest extends ProducerConsumerBase { + private static final Logger log = LoggerFactory.getLogger(KeySharedSubscriptionDisabledBrokerCacheTest.class); + private static final List keys = Arrays.asList("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"); + private static final String SUBSCRIPTION_NAME = "key_shared"; + private final KeySharedImplementationType implementationType; + + // Comment out the next line (Factory annotation) to run tests manually in IntelliJ, one-by-one + @Factory + public static Object[] createTestInstances() { + return KeySharedImplementationType.generateTestInstances(KeySharedSubscriptionDisabledBrokerCacheTest::new); + } + + public KeySharedSubscriptionDisabledBrokerCacheTest() { + // set the default implementation type for manual running in IntelliJ + this(KeySharedImplementationType.PIP379); + } + + public KeySharedSubscriptionDisabledBrokerCacheTest(KeySharedImplementationType implementationType) { + this.implementationType = implementationType; + } + + @DataProvider(name = "currentImplementationType") + public Object[] currentImplementationType() { + return new Object[]{ implementationType }; + } + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(implementationType.classic); + conf.setSubscriptionSharedUseClassicPersistentImplementation(implementationType.classic); + this.conf.setUnblockStuckSubscriptionEnabled(false); + this.conf.setSubscriptionKeySharedUseConsistentHashing(true); + conf.setManagedLedgerCacheSizeMB(0); + conf.setManagedLedgerMaxReadsInFlightSizeInMB(0); + conf.setDispatcherRetryBackoffInitialTimeInMs(0); + conf.setDispatcherRetryBackoffMaxTimeInMs(0); + conf.setKeySharedUnblockingIntervalMs(0); + conf.setBrokerDeduplicationEnabled(true); + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @AfterMethod(alwaysRun = true) + public void resetDefaultNamespace() throws Exception { + List list = admin.namespaces().getTopics("public/default"); + for (String topicName : list){ + if (!pulsar.getBrokerService().isSystemTopic(topicName)) { + admin.topics().delete(topicName, false); + } + } + // reset read ahead limits to defaults + ServiceConfiguration defaultConf = new ServiceConfiguration(); + conf.setKeySharedLookAheadMsgInReplayThresholdPerSubscription( + defaultConf.getKeySharedLookAheadMsgInReplayThresholdPerSubscription()); + conf.setKeySharedLookAheadMsgInReplayThresholdPerConsumer( + defaultConf.getKeySharedLookAheadMsgInReplayThresholdPerConsumer()); + } + + // Use a fixed seed to make the tests using random values deterministic + // When a test fails, it's possible to re-run it to reproduce the issue + private static final Random random = new Random(1); + + private Producer createProducer(String topic, boolean enableBatch) throws PulsarClientException { + Producer producer = null; + if (enableBatch) { + producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .enableBatching(true) + .maxPendingMessages(2001) + .batcherBuilder(BatcherBuilder.KEY_BASED) + .create(); + } else { + producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .maxPendingMessages(2001) + .enableBatching(false) + .create(); + } + return producer; + } + + @SneakyThrows + private StickyKeyConsumerSelector getSelector(String topic, String subscription) { + Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); + StickyKeyDispatcher dispatcher = (StickyKeyDispatcher) sub.getDispatcher(); + return dispatcher.getSelector(); + } + + @Test(dataProvider = "currentImplementationType", invocationCount = 1) + public void testMessageOrderInSingleConsumerReconnect(KeySharedImplementationType impl) throws Exception { + String topic = newUniqueName("testMessageOrderInSingleConsumerReconnect"); + int numberOfKeys = 100; + long pauseTime = 100L; + // don't fail if duplicates are out-of-order + // it's possible to change this setting while experimenting + boolean failOnDuplicatesOutOfOrder = false; + + @Cleanup + PulsarClient pulsarClient2 = PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .build(); + + @Cleanup + PulsarClient pulsarClient3 = PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .build(); + + @Cleanup + Producer producer = createProducer(topic, false); + + // create a consumer and close it to create a subscription + pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe() + .close(); + + Set remainingMessageValues = Collections.synchronizedSet(new HashSet<>()); + BlockingQueue, Message>> unackedMessages = new LinkedBlockingQueue<>(); + AtomicBoolean c2MessagesShouldBeUnacked = new AtomicBoolean(true); + Set keysForC2 = new HashSet<>(); + AtomicLong lastMessageTimestamp = new AtomicLong(System.currentTimeMillis()); + List exceptionsInHandler = Collections.synchronizedList(new ArrayList<>()); + + Map> keyPositions = new HashMap<>(); + MessageListener messageHandler = (consumer, msg) -> { + lastMessageTimestamp.set(System.currentTimeMillis()); + synchronized (this) { + try { + String key = msg.getKey(); + if (c2MessagesShouldBeUnacked.get() && keysForC2.contains(key)) { + unackedMessages.add(Pair.of(consumer, msg)); + return; + } + long delayMillis = ThreadLocalRandom.current().nextLong(25, 50); + CompletableFuture.delayedExecutor(delayMillis, TimeUnit.MILLISECONDS).execute(() -> + consumer.acknowledgeAsync(msg)); + MessageIdAdv msgId = (MessageIdAdv) msg.getMessageId(); + Position currentPosition = PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId()); + Pair prevPair = keyPositions.get(key); + if (prevPair != null && prevPair.getLeft().compareTo(currentPosition) > 0) { + boolean isDuplicate = !remainingMessageValues.contains(msg.getValue()); + String errorMessage = String.format( + "out of order: key: %s value: %s prev: %s/%s current: %s/%s duplicate: %s", + key, msg.getValue(), + prevPair.getLeft(), prevPair.getRight(), + currentPosition, consumer.getConsumerName(), isDuplicate); + log.error(errorMessage); + if (!isDuplicate || failOnDuplicatesOutOfOrder) { + fail(errorMessage); + } + } + keyPositions.put(key, Pair.of(currentPosition, consumer.getConsumerName())); + boolean removed = remainingMessageValues.remove(msg.getValue()); + if (!removed) { + // duplicates are possible during reconnects, this is not an error + log.warn("Duplicate message: {} value: {}", msg.getMessageId(), msg.getValue()); + } + } catch (Throwable t) { + exceptionsInHandler.add(t); + if (!(t instanceof AssertionError)) { + log.error("Error in message handler", t); + } + } + } + }; + + // Adding a new consumer. + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .messageListener(messageHandler) + .subscribe(); + + @Cleanup + Consumer c2 = pulsarClient2.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .messageListener(messageHandler) + .subscribe(); + + @Cleanup + Consumer c3 = pulsarClient3.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .messageListener(messageHandler) + .subscribe(); + + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + + // find keys that will be assigned to c2 + for (int i = 0; i < numberOfKeys; i++) { + String key = String.valueOf(i); + byte[] keyBytes = key.getBytes(UTF_8); + int hash = selector.makeStickyKeyHash(keyBytes); + if (selector.select(hash).consumerName().equals("c2")) { + keysForC2.add(key); + } + } + + // close c2 + c2.close(); + Thread.sleep(pauseTime); + + // produce messages with random keys + for (int i = 0; i < 1000; i++) { + String key = String.valueOf(random.nextInt(numberOfKeys)); + //log.info("Producing message with key: {} value: {}", key, i); + remainingMessageValues.add(i); + producer.newMessage() + .key(key) + .value(i) + .send(); + } + + // reconnect c2 + c2 = pulsarClient2.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .messageListener(messageHandler) + .startPaused(true) + .subscribe(); + + Thread.sleep(2 * pauseTime); + + // produce messages with c2 keys so that possible race conditions would be more likely to happen + List keysForC2List=new ArrayList<>(keysForC2); + for (int i = 1000; i < 1100; i++) { + String key = keysForC2List.get(random.nextInt(keysForC2List.size())); + log.info("Producing message with key: {} value: {}", key, i); + remainingMessageValues.add(i); + producer.newMessage() + .key(key) + .value(i) + .send(); + } + + Thread.sleep(2 * pauseTime); + + log.info("Acking unacked messages to unblock c2 keys"); + // ack the unacked messages to unblock c2 keys + c2MessagesShouldBeUnacked.set(false); + Pair, Message> consumerMessagePair; + while ((consumerMessagePair = unackedMessages.poll()) != null) { + messageHandler.received(consumerMessagePair.getLeft(), consumerMessagePair.getRight()); + } + + // resume c2 so that permits are while hashes are unblocked so that possible race conditions would + // be more likely to happen + log.info("Resuming c2"); + c2.resume(); + + Awaitility.await().atMost(Duration.ofSeconds(10)).until(() -> { + return remainingMessageValues.isEmpty() + || System.currentTimeMillis() - lastMessageTimestamp.get() > 50 * pauseTime; + }); + + try { + assertSoftly(softly -> { + softly.assertThat(remainingMessageValues).as("remainingMessageValues").isEmpty(); + softly.assertThat(exceptionsInHandler).as("exceptionsInHandler").isEmpty(); + }); + } finally { + logTopicStats(topic); + } + } +} diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml index a0732096f2845..7b3cd6a04fcca 100644 --- a/pulsar-broker/src/test/resources/log4j2.xml +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -40,12 +40,18 @@ + + + + + + - --> + --> From 6d8d73df727cceb8db265d9e33455ee5c52a8791 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 23 Jan 2025 13:40:03 +0200 Subject: [PATCH 269/327] [improve][ci] Increase Maven max heap size to 2048M and tune GCLockerRetryAllocationCount (#23883) --- .github/workflows/ci-owasp-dependency-check.yaml | 2 +- .github/workflows/pulsar-ci-flaky.yaml | 2 +- .github/workflows/pulsar-ci.yaml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/ci-owasp-dependency-check.yaml b/.github/workflows/ci-owasp-dependency-check.yaml index 6e60065b07ba9..4a2e12a06377a 100644 --- a/.github/workflows/ci-owasp-dependency-check.yaml +++ b/.github/workflows/ci-owasp-dependency-check.yaml @@ -24,7 +24,7 @@ on: workflow_dispatch: env: - MAVEN_OPTS: -Xss1500k -Xmx1500m -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000 + MAVEN_OPTS: -Xss1500k -Xmx2048m -XX:+UnlockDiagnosticVMOptions -XX:GCLockerRetryAllocationCount=100 -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000 JDK_DISTRIBUTION: corretto NIST_NVD_API_KEY: ${{ secrets.NIST_NVD_API_KEY }} diff --git a/.github/workflows/pulsar-ci-flaky.yaml b/.github/workflows/pulsar-ci-flaky.yaml index 514334d123c44..c1d0e3ae3f539 100644 --- a/.github/workflows/pulsar-ci-flaky.yaml +++ b/.github/workflows/pulsar-ci-flaky.yaml @@ -65,7 +65,7 @@ concurrency: cancel-in-progress: true env: - MAVEN_OPTS: -Xss1500k -Xmx1500m -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000 + MAVEN_OPTS: -Xss1500k -Xmx2048m -XX:+UnlockDiagnosticVMOptions -XX:GCLockerRetryAllocationCount=100 -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000 # defines the retention period for the intermediate build artifacts needed for rerunning a failed build job # it's possible to rerun individual failed jobs when the build artifacts are available # if the artifacts have already been expired, the complete workflow can be rerun by closing and reopening the PR or by rebasing the PR diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 6c71efe7e8549..c004b61c2e362 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -65,7 +65,7 @@ concurrency: cancel-in-progress: true env: - MAVEN_OPTS: -Xss1500k -Xmx1500m -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000 + MAVEN_OPTS: -Xss1500k -Xmx2048m -XX:+UnlockDiagnosticVMOptions -XX:GCLockerRetryAllocationCount=100 -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000 # defines the retention period for the intermediate build artifacts needed for rerunning a failed build job # it's possible to rerun individual failed jobs when the build artifacts are available # if the artifacts have already been expired, the complete workflow can be rerun by closing and reopening the PR or by rebasing the PR From 0f9f66181fade7b5163372591d2b299b6ba31780 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 23 Jan 2025 16:02:45 +0200 Subject: [PATCH 270/327] [improve][broker] Support values up to 2^32 in ConcurrentBitmapSortedLongPairSet (#23878) --- .../ConcurrentBitmapSortedLongPairSet.java | 14 ++++++--- ...ConcurrentBitmapSortedLongPairSetTest.java | 30 +++++++++++++++++-- 2 files changed, 37 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java index 7a4126fedec64..70437d07dbee0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.utils; -import java.util.Iterator; import java.util.Map; import java.util.NavigableMap; import java.util.NavigableSet; @@ -29,8 +28,13 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.lang3.mutable.MutableObject; import org.apache.pulsar.common.util.collections.LongPairSet; +import org.roaringbitmap.PeekableIntIterator; import org.roaringbitmap.RoaringBitmap; +/** + * A concurrent set of pairs of longs. + * The right side of the value supports unsigned values up to 2^32. + */ public class ConcurrentBitmapSortedLongPairSet { private final NavigableMap map = new TreeMap<>(); @@ -139,10 +143,12 @@ public > void processItems(LongPairSet.LongPairFunction< lock.readLock().lock(); try { for (Map.Entry entry : map.entrySet()) { - Iterator iterator = entry.getValue().stream().iterator(); + PeekableIntIterator intIterator = entry.getValue().getIntIterator(); boolean continueProcessing = true; - while (continueProcessing && iterator.hasNext()) { - T item = longPairConverter.apply(entry.getKey(), iterator.next()); + while (continueProcessing && intIterator.hasNext()) { + // RoaringBitmap encodes values as unsigned 32-bit integers internally, it's necessary to use + // Integer.toUnsignedLong to convert them to unsigned long values + T item = longPairConverter.apply(entry.getKey(), Integer.toUnsignedLong(intIterator.next())); continueProcessing = itemProcessor.process(item); } if (!continueProcessing) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSetTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSetTest.java index 5f8f13288cfe8..34f971e8841ab 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSetTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSetTest.java @@ -18,18 +18,19 @@ */ package org.apache.pulsar.utils; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; -import lombok.Cleanup; -import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet; -import org.testng.annotations.Test; import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import lombok.Cleanup; +import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet; +import org.testng.annotations.Test; @Test(groups = "utils") public class ConcurrentBitmapSortedLongPairSetTest { @@ -204,4 +205,27 @@ public void concurrentInsertions() throws Throwable { assertEquals(set.size(), N * nThreads); } + + @Test + public void testValueLargerThanIntegerMAX_VALUE() { + ConcurrentBitmapSortedLongPairSet set = new ConcurrentBitmapSortedLongPairSet(); + long baseValue = Integer.MAX_VALUE; + List addedValues = new ArrayList<>(); + int items = 10; + for (int i = 0; i < items; i++) { + long value = baseValue + i; + set.add(1, value); + addedValues.add(value); + } + assertEquals(set.size(), items); + List values = new ArrayList<>(); + set.processItems((item1, item2) -> { + assertEquals(item1, 1); + return item2; + }, (value) -> { + values.add(value); + return true; + }); + assertThat(values).containsExactlyElementsOf(addedValues); + } } From 66d1bb0d734f12d758b0f0e9e3c0b42543508f8d Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Fri, 24 Jan 2025 12:10:59 +0800 Subject: [PATCH 271/327] [improve][fn] Set default tenant and namespace for ListFunctions cmd (#23881) --- .../apache/pulsar/admin/cli/CmdFunctionsTest.java | 13 +++++++++++++ .../org/apache/pulsar/admin/cli/CmdFunctions.java | 10 ++++++++++ 2 files changed, 23 insertions(+) diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java index d3087b7fc873c..5cac07502b47f 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java @@ -631,6 +631,19 @@ public void testListFunctions() throws Exception { verify(functions, times(1)).getFunctions(eq(TENANT), eq(NAMESPACE)); } + @Test + public void testListFunctionsWithDefaultValue() throws Exception { + cmd.run(new String[] { + "list", + }); + + ListFunctions lister = cmd.getLister(); + assertEquals("public", lister.getTenant()); + assertEquals("default", lister.getNamespace()); + + verify(functions, times(1)).getFunctions(eq("public"), eq("default")); + } + @Test public void testStateGetter() throws Exception { String key = TEST_NAME + "-key"; diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java index 4c7e058af6de1..a1b8d26ef1883 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java @@ -105,6 +105,16 @@ abstract class NamespaceCommand extends BaseCommand { @Option(names = "--namespace", description = "The namespace of a Pulsar Function") protected String namespace; + + @Override + public void processArguments() { + if (tenant == null) { + tenant = PUBLIC_TENANT; + } + if (namespace == null) { + namespace = DEFAULT_NAMESPACE; + } + } } /** From 798a014ab7d1bd1055a9f7a62cfd4ed0120d798a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 24 Jan 2025 19:51:50 +0800 Subject: [PATCH 272/327] [improve][ml]Set default value of managedLedgerPersistIndividualAckAsLongArray to true (#23888) --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 00e1113adfb9f..ce3b25571e271 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2269,7 +2269,7 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece @FieldContext( category = CATEGORY_STORAGE_ML, doc = "Whether persist cursor ack stats as long arrays, which will compress the data and reduce GC rate") - private boolean managedLedgerPersistIndividualAckAsLongArray = false; + private boolean managedLedgerPersistIndividualAckAsLongArray = true; @FieldContext( category = CATEGORY_STORAGE_ML, doc = "If enabled, the maximum \"acknowledgment holes\" will not be limited and \"acknowledgment holes\" " From 223eea027beb1a7e95ff4519db181268b4636829 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Sat, 25 Jan 2025 02:44:16 +0800 Subject: [PATCH 273/327] [fix][client] Fix LoadManagerReport not found (#23886) --- pulsar-client-admin-shaded/pom.xml | 7 +++++++ pulsar-client-all/pom.xml | 7 +++++++ pulsar-client-shaded/pom.xml | 7 +++++++ .../pulsar/common/util/ObjectMapperFactory.java | 11 +---------- .../apache/pulsar/tests/integration/SmokeTest.java | 1 + .../apache/pulsar/tests/integration/SmokeTest.java | 1 + 6 files changed, 24 insertions(+), 10 deletions(-) diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index de54c3d049639..790691878f69c 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -367,6 +367,13 @@ org.apache.pulsar.policies org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport + org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats + org.apache.pulsar.policies.data.loadbalancer.ResourceUsage + org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData + org.asynchttpclient diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 4fec9ff51b894..322885892a5f4 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -427,6 +427,13 @@ org.apache.pulsar.policies org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport + org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats + org.apache.pulsar.policies.data.loadbalancer.ResourceUsage + org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData + org.asynchttpclient diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index d8adacbe8a0bb..66032b19fbce4 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -339,6 +339,13 @@ org.apache.pulsar.policies org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport + org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats + org.apache.pulsar.policies.data.loadbalancer.ResourceUsage + org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData + org.asynchttpclient diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java index b737d68d5ea9f..33bc09725f62f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java @@ -32,7 +32,6 @@ import com.fasterxml.jackson.module.paramnames.ParameterNamesModule; import java.util.concurrent.atomic.AtomicReference; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.ClassUtils; import org.apache.pulsar.client.admin.internal.data.AuthPoliciesImpl; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.FunctionState; @@ -263,15 +262,7 @@ private static void setAnnotationsModule(ObjectMapper mapper) { mapper.addMixIn(FunctionState.class, JsonIgnorePropertiesMixIn.class); mapper.addMixIn(Metrics.class, MetricsMixIn.class); - try { - // We look for LoadManagerReport first, then add deserializer to the module - // With shaded client, org.apache.pulsar.policies is relocated to - // org.apache.pulsar.shade.org.apache.pulsar.policies - ClassUtils.getClass("org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport"); - module.addDeserializer(LoadManagerReport.class, new LoadReportDeserializer()); - } catch (ClassNotFoundException e) { - log.debug("Add LoadManagerReport deserializer failed because LoadManagerReport.class has been shaded", e); - } + module.addDeserializer(LoadManagerReport.class, new LoadReportDeserializer()); module.setAbstractTypes(resolver); diff --git a/tests/pulsar-client-admin-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java b/tests/pulsar-client-admin-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java index b5c615b743cf7..0c4ffb283f5e7 100644 --- a/tests/pulsar-client-admin-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java +++ b/tests/pulsar-client-admin-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java @@ -87,6 +87,7 @@ public void checkAdmin() throws PulsarClientException, PulsarAdminException { expectedNamespacesList.add("public/default"); expectedNamespacesList.add("public/functions"); Assert.assertEquals(admin.namespaces().getNamespaces("public"), expectedNamespacesList); + admin.brokerStats().getLoadReport(); } @Override diff --git a/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java b/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java index c5075118f08e3..bd2282703d804 100644 --- a/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java +++ b/tests/pulsar-client-all-shade-test/src/test/java/org/apache/pulsar/tests/integration/SmokeTest.java @@ -87,6 +87,7 @@ public void checkAdmin() throws PulsarClientException, PulsarAdminException { expectedNamespacesList.add("public/default"); expectedNamespacesList.add("public/functions"); Assert.assertEquals(admin.namespaces().getNamespaces("public"), expectedNamespacesList); + admin.brokerStats().getLoadReport(); } @Override From 2a9d4ac85d8d786979afaa0b965cdb27375ae969 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 25 Jan 2025 10:59:01 +0200 Subject: [PATCH 274/327] [improve][test] Support decorating topic, subscription, dispatcher, ManagedLedger and ManagedCursors instances in tests (#23892) --- .../impl/ManagedLedgerFactoryImpl.java | 15 +- .../mledger/impl/ManagedLedgerImpl.java | 10 +- .../broker/ManagedLedgerClientFactory.java | 14 +- .../pulsar/broker/service/TopicFactory.java | 5 + .../persistent/PersistentSubscription.java | 139 ++++++------ .../service/persistent/PersistentTopic.java | 12 +- .../BrokerTestInterceptor.java | 212 ++++++++++++++++++ 7 files changed, 333 insertions(+), 74 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/testinterceptor/BrokerTestInterceptor.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index f546a487f84be..12c3ea12df581 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -410,11 +410,8 @@ public void asyncOpen(final String name, final ManagedLedgerConfig config, final new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), config.getBookKeeperEnsemblePlacementPolicyProperties())) .thenAccept(bk -> { - final ManagedLedgerImpl newledger = config.getShadowSource() == null - ? new ManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name, - mlOwnershipChecker) - : new ShadowManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name, - mlOwnershipChecker); + final ManagedLedgerImpl newledger = + createManagedLedger(bk, store, name, config, mlOwnershipChecker); PendingInitializeManagedLedger pendingLedger = new PendingInitializeManagedLedger(newledger); pendingInitializeLedgers.put(name, pendingLedger); newledger.initialize(new ManagedLedgerInitializeLedgerCallback() { @@ -472,6 +469,14 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { }); } + protected ManagedLedgerImpl createManagedLedger(BookKeeper bk, MetaStore store, String name, + ManagedLedgerConfig config, + Supplier> mlOwnershipChecker) { + return config.getShadowSource() == null + ? new ManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name, mlOwnershipChecker) : + new ShadowManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name, mlOwnershipChecker); + } + @Override public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback, 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 966aa068f2ff5..4f45fc67b6377 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 @@ -632,7 +632,7 @@ public void operationComplete(List consumers, Stat s) { for (final String cursorName : consumers) { log.info("[{}] Loading cursor {}", name, cursorName); final ManagedCursorImpl cursor; - cursor = new ManagedCursorImpl(bookKeeper, ManagedLedgerImpl.this, cursorName); + cursor = createCursor(ManagedLedgerImpl.this.bookKeeper, cursorName); cursor.recover(new VoidCallback() { @Override @@ -663,7 +663,7 @@ public void operationFailed(ManagedLedgerException exception) { log.debug("[{}] Recovering cursor {} lazily", name, cursorName); } final ManagedCursorImpl cursor; - cursor = new ManagedCursorImpl(bookKeeper, ManagedLedgerImpl.this, cursorName); + cursor = createCursor(ManagedLedgerImpl.this.bookKeeper, cursorName); CompletableFuture cursorRecoveryFuture = new CompletableFuture<>(); uninitializedCursors.put(cursorName, cursorRecoveryFuture); @@ -1007,7 +1007,7 @@ public synchronized void asyncOpenCursor(final String cursorName, final InitialP if (log.isDebugEnabled()) { log.debug("[{}] Creating new cursor: {}", name, cursorName); } - final ManagedCursorImpl cursor = new ManagedCursorImpl(bookKeeper, this, cursorName); + final ManagedCursorImpl cursor = createCursor(bookKeeper, cursorName); CompletableFuture cursorFuture = new CompletableFuture<>(); uninitializedCursors.put(cursorName, cursorFuture); Position position = InitialPosition.Earliest == initialPosition ? getFirstPosition() : getLastPosition(); @@ -1039,6 +1039,10 @@ public void operationFailed(ManagedLedgerException exception) { }); } + protected ManagedCursorImpl createCursor(BookKeeper bookKeeper, String cursorName) { + return new ManagedCursorImpl(bookKeeper, this, cursorName); + } + @Override public synchronized void asyncDeleteCursor(final String consumerName, final DeleteCursorCallback callback, final Object ctx) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java index 737bc69bf24df..3d945afe4c115 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java @@ -116,8 +116,8 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata try { this.managedLedgerFactory = - new ManagedLedgerFactoryImpl(metadataStore, bkFactory, managedLedgerFactoryConfig, statsLogger, - openTelemetry); + createManagedLedgerFactory(metadataStore, openTelemetry, bkFactory, managedLedgerFactoryConfig, + statsLogger); } catch (Exception e) { statsProvider.stop(); defaultBkClient.close(); @@ -147,6 +147,16 @@ public BookKeeper getBookKeeperClient() { }; } + protected ManagedLedgerFactoryImpl createManagedLedgerFactory(MetadataStoreExtended metadataStore, + OpenTelemetry openTelemetry, + BookkeeperFactoryForCustomEnsemblePlacementPolicy + bkFactory, + ManagedLedgerFactoryConfig managedLedgerFactoryConfig, + StatsLogger statsLogger) throws Exception { + return new ManagedLedgerFactoryImpl(metadataStore, bkFactory, managedLedgerFactoryConfig, statsLogger, + openTelemetry); + } + @Override public Collection getStorageClasses() { return List.of(getDefaultStorageClass()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicFactory.java index f8bac9d8134b0..523f995cc5dc9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicFactory.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import java.io.Closeable; +import java.io.IOException; import org.apache.bookkeeper.mledger.ManagedLedger; /** @@ -28,4 +29,8 @@ public interface TopicFactory extends Closeable { T create(String topic, ManagedLedger ledger, BrokerService brokerService, Class topicClazz); + + default void close() throws IOException { + // default implementation + } } \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index a96a7e75506eb..275d1ae5818b0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -250,70 +250,10 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { } if (dispatcher == null || !dispatcher.isConsumerConnected()) { - Dispatcher previousDispatcher = null; - switch (consumer.subType()) { - case Exclusive: - if (dispatcher == null || dispatcher.getType() != SubType.Exclusive) { - previousDispatcher = dispatcher; - dispatcher = new PersistentDispatcherSingleActiveConsumer( - cursor, SubType.Exclusive, 0, topic, this); - } - break; - case Shared: - if (dispatcher == null || dispatcher.getType() != SubType.Shared) { - previousDispatcher = dispatcher; - if (config.isSubscriptionSharedUseClassicPersistentImplementation()) { - dispatcher = new PersistentDispatcherMultipleConsumersClassic(topic, cursor, this); - } else { - dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, this); - } - } - break; - case Failover: - int partitionIndex = TopicName.getPartitionIndex(topicName); - if (partitionIndex < 0) { - // For non partition topics, use a negative index so - // dispatcher won't sort consumers before picking - // an active consumer for the topic. - partitionIndex = -1; - } - - if (dispatcher == null || dispatcher.getType() != SubType.Failover) { - previousDispatcher = dispatcher; - dispatcher = new PersistentDispatcherSingleActiveConsumer(cursor, SubType.Failover, - partitionIndex, topic, this); - } - break; - case Key_Shared: - KeySharedMeta ksm = consumer.getKeySharedMeta(); - if (dispatcher == null || dispatcher.getType() != SubType.Key_Shared - || !((StickyKeyDispatcher) dispatcher) - .hasSameKeySharedPolicy(ksm)) { - previousDispatcher = dispatcher; - if (config.isSubscriptionKeySharedUseClassicPersistentImplementation()) { - dispatcher = - new PersistentStickyKeyDispatcherMultipleConsumersClassic(topic, cursor, - this, - topic.getBrokerService().getPulsar().getConfiguration(), ksm); - } else { - dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this, - topic.getBrokerService().getPulsar().getConfiguration(), ksm); - } - } - break; - default: - return FutureUtil.failedFuture( - new ServerMetadataException("Unsupported subscription type")); - } - - if (previousDispatcher != null) { - previousDispatcher.close().thenRun(() -> { - log.info("[{}][{}] Successfully closed previous dispatcher", topicName, subName); - }).exceptionally(ex -> { - log.error("[{}][{}] Failed to close previous dispatcher", topicName, subName, ex); - return null; - }); + if (consumer.subType() == null) { + return FutureUtil.failedFuture(new ServerMetadataException("Unsupported subscription type")); } + dispatcher = reuseOrCreateDispatcher(dispatcher, consumer); } else { Optional> compatibilityError = checkForConsumerCompatibilityErrorWithDispatcher(dispatcher, consumer); @@ -327,6 +267,79 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { }); } + /** + * Create a new dispatcher or reuse the existing one when it's compatible with the new consumer. + * This protected method can be overridded for testing purpose for injecting test dispatcher instances with + * special behaviors. + * @param dispatcher the existing dispatcher + * @param consumer the new consumer + * @return the dispatcher to use, either the existing one or a new one + */ + protected Dispatcher reuseOrCreateDispatcher(Dispatcher dispatcher, Consumer consumer) { + Dispatcher previousDispatcher = null; + switch (consumer.subType()) { + case Exclusive: + if (dispatcher == null || dispatcher.getType() != SubType.Exclusive) { + previousDispatcher = dispatcher; + dispatcher = new PersistentDispatcherSingleActiveConsumer( + cursor, SubType.Exclusive, 0, topic, this); + } + break; + case Shared: + if (dispatcher == null || dispatcher.getType() != SubType.Shared) { + previousDispatcher = dispatcher; + if (config.isSubscriptionSharedUseClassicPersistentImplementation()) { + dispatcher = new PersistentDispatcherMultipleConsumersClassic(topic, cursor, this); + } else { + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, this); + } + } + break; + case Failover: + int partitionIndex = TopicName.getPartitionIndex(topicName); + if (partitionIndex < 0) { + // For non partition topics, use a negative index so + // dispatcher won't sort consumers before picking + // an active consumer for the topic. + partitionIndex = -1; + } + + if (dispatcher == null || dispatcher.getType() != SubType.Failover) { + previousDispatcher = dispatcher; + dispatcher = new PersistentDispatcherSingleActiveConsumer(cursor, SubType.Failover, + partitionIndex, topic, this); + } + break; + case Key_Shared: + KeySharedMeta ksm = consumer.getKeySharedMeta(); + if (dispatcher == null || dispatcher.getType() != SubType.Key_Shared + || !((StickyKeyDispatcher) dispatcher) + .hasSameKeySharedPolicy(ksm)) { + previousDispatcher = dispatcher; + if (config.isSubscriptionKeySharedUseClassicPersistentImplementation()) { + dispatcher = + new PersistentStickyKeyDispatcherMultipleConsumersClassic(topic, cursor, + this, config, ksm); + } else { + dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this, + config, ksm); + } + } + break; + } + + if (previousDispatcher != null) { + previousDispatcher.close().thenRun(() -> { + log.info("[{}][{}] Successfully closed previous dispatcher", topicName, subName); + }).exceptionally(ex -> { + log.error("[{}][{}] Failed to close previous dispatcher", topicName, subName, ex); + return null; + }); + } + + return dispatcher; + } + @Override public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor) throws BrokerServiceException { cursor.updateLastActive(); 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 2325c8286a1be..e920c483bb3ea 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 @@ -586,7 +586,17 @@ public CompletableFuture unloadSubscription(@Nonnull String subName) { }); } - private PersistentSubscription createPersistentSubscription(String subscriptionName, ManagedCursor cursor, + + /** + * Create a new subscription instance for the topic. + * This protected method can be overridden in tests to return a special test implementation instance. + * @param subscriptionName the name of the subscription + * @param cursor the cursor to use for the subscription + * @param replicated the subscription replication flag + * @param subscriptionProperties the subscription properties + * @return the subscription instance + */ + protected PersistentSubscription createPersistentSubscription(String subscriptionName, ManagedCursor cursor, Boolean replicated, Map subscriptionProperties) { requireNonNull(topicCompactionService); if (isCompactionSubscription(subscriptionName) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testinterceptor/BrokerTestInterceptor.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testinterceptor/BrokerTestInterceptor.java new file mode 100644 index 0000000000000..a1549b2cb86b3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testinterceptor/BrokerTestInterceptor.java @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.testinterceptor; + +import io.opentelemetry.api.OpenTelemetry; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import java.util.function.Supplier; +import lombok.Getter; +import lombok.Setter; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.MetaStore; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ManagedLedgerClientFactory; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.TopicFactory; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; + +/** + * A test interceptor for broker tests that allows to decorate persistent topics, subscriptions, dispatchers + * managed ledger factory, managed ledger and managed cursor instances. + */ +public class BrokerTestInterceptor { + public static final BrokerTestInterceptor INSTANCE = new BrokerTestInterceptor(); + + // Suppress default constructor for noninstantiability + private BrokerTestInterceptor() { + + } + + public static class TestTopicFactory implements TopicFactory { + @Override + public T create(String topic, ManagedLedger ledger, BrokerService brokerService, + Class topicClazz) { + if (!topicClazz.isAssignableFrom(PersistentTopic.class)) { + throw new UnsupportedOperationException("Unsupported topic class"); + } + return topicClazz.cast( + INSTANCE.getPersistentTopicDecorator().apply(new TestTopic(topic, ledger, brokerService))); + } + } + + static class TestTopic extends PersistentTopic { + + public TestTopic(String topic, ManagedLedger ledger, BrokerService brokerService) { + super(topic, ledger, brokerService); + } + + @Override + protected PersistentSubscription createPersistentSubscription(String subscriptionName, ManagedCursor cursor, + Boolean replicated, + Map subscriptionProperties) { + return INSTANCE.getPersistentSubscriptionDecorator() + .apply(new TestSubscription(this, subscriptionName, cursor, replicated, subscriptionProperties)); + } + } + + static class TestSubscription extends PersistentSubscription { + public TestSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, + Boolean replicated, + Map subscriptionProperties) { + super(topic, subscriptionName, cursor, replicated, subscriptionProperties); + } + + @Override + protected Dispatcher reuseOrCreateDispatcher(Dispatcher dispatcher, + Consumer consumer) { + Dispatcher previousInstance = dispatcher; + dispatcher = super.reuseOrCreateDispatcher(dispatcher, consumer); + if (dispatcher != previousInstance) { + dispatcher = INSTANCE.getDispatcherDecorator().apply(dispatcher); + } + return dispatcher; + } + } + + public static class TestManagedLedgerStorage extends ManagedLedgerClientFactory { + @Override + protected ManagedLedgerFactoryImpl createManagedLedgerFactory(MetadataStoreExtended metadataStore, + OpenTelemetry openTelemetry, + ManagedLedgerFactoryImpl.BookkeeperFactoryForCustomEnsemblePlacementPolicy bkFactory, + ManagedLedgerFactoryConfig managedLedgerFactoryConfig, + StatsLogger statsLogger) throws Exception { + return INSTANCE.managedLedgerFactoryDecorator.apply( + new TestManagedLedgerFactoryImpl(metadataStore, bkFactory, managedLedgerFactoryConfig, statsLogger, + openTelemetry)); + } + } + + static class TestManagedLedgerFactoryImpl extends ManagedLedgerFactoryImpl { + public TestManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, + BookkeeperFactoryForCustomEnsemblePlacementPolicy bookKeeperGroupFactory, + ManagedLedgerFactoryConfig config, StatsLogger statsLogger, + OpenTelemetry openTelemetry) throws Exception { + super(metadataStore, bookKeeperGroupFactory, config, statsLogger, openTelemetry); + } + + @Override + protected ManagedLedgerImpl createManagedLedger(BookKeeper bk, MetaStore store, String name, + ManagedLedgerConfig config, + Supplier> mlOwnershipChecker) { + return INSTANCE.managedLedgerDecorator.apply( + new TestManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name, mlOwnershipChecker)); + } + } + + static class TestManagedLedgerImpl extends ManagedLedgerImpl { + public TestManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, + ManagedLedgerConfig config, + OrderedScheduler scheduledExecutor, String name, + Supplier> mlOwnershipChecker) { + super(factory, bookKeeper, store, config, scheduledExecutor, name, mlOwnershipChecker); + } + + @Override + protected ManagedCursorImpl createCursor(BookKeeper bookKeeper, String cursorName) { + return INSTANCE.managedCursorDecorator.apply(super.createCursor(bookKeeper, cursorName)); + } + } + + @Getter + @Setter + private Function persistentTopicDecorator = Function.identity(); + + @Getter + @Setter + private Function persistentSubscriptionDecorator = Function.identity(); + + @Getter + @Setter + private Function dispatcherDecorator = Function.identity(); + + @Getter + @Setter + private Function managedLedgerFactoryDecorator = Function.identity(); + + @Getter + @Setter + private Function managedLedgerDecorator = Function.identity(); + + @Getter + @Setter + private Function managedCursorDecorator = Function.identity(); + + public void reset() { + persistentTopicDecorator = Function.identity(); + persistentSubscriptionDecorator = Function.identity(); + dispatcherDecorator = Function.identity(); + managedLedgerFactoryDecorator = Function.identity(); + managedLedgerDecorator = Function.identity(); + managedCursorDecorator = Function.identity(); + } + + public void configure(ServiceConfiguration conf) { + conf.setTopicFactoryClassName(TestTopicFactory.class.getName()); + conf.setManagedLedgerStorageClassName(TestManagedLedgerStorage.class.getName()); + } + + public void applyDispatcherSpyDecorator(Class dispatcherClass, + java.util.function.Consumer spyCustomizer) { + setDispatcherDecorator(createDispatcherSpyDecorator(dispatcherClass, spyCustomizer)); + } + + public static Function createDispatcherSpyDecorator( + Class dispatcherClass, java.util.function.Consumer spyCustomizer) { + return dispatcher -> { + Dispatcher spy = BrokerTestUtil.spyWithoutRecordingInvocations(dispatcher); + spyCustomizer.accept(dispatcherClass.cast(spy)); + return spy; + }; + } + + public void applyCursorSpyDecorator(java.util.function.Consumer spyCustomizer) { + setManagedCursorDecorator(cursor -> { + ManagedCursorImpl spy = BrokerTestUtil.spyWithoutRecordingInvocations(cursor); + spyCustomizer.accept(spy); + return spy; + }); + } +} From 11a615e7cecb57c8c240cc565c5ac62d938c490d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 27 Jan 2025 08:10:54 +0200 Subject: [PATCH 275/327] [fix][broker] Apply dispatcherMaxReadSizeBytes also for replay reads for Shared and Key_Shared subscriptions (#23894) --- ...PersistentDispatcherMultipleConsumers.java | 13 +- ...atcherMultipleConsumersReadLimitsTest.java | 144 ++++++++++++++++++ ...ckyKeyDispatcherMultipleConsumersTest.java | 4 + 3 files changed, 157 insertions(+), 4 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersReadLimitsTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 82b96c365072f..3ceb703c26cd8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -384,7 +384,7 @@ public synchronized void readMoreEntries() { } Set messagesToReplayNow = - canReplayMessages() ? getMessagesToReplayNow(messagesToRead) : Collections.emptySet(); + canReplayMessages() ? getMessagesToReplayNow(messagesToRead, bytesToRead) : Collections.emptySet(); if (!messagesToReplayNow.isEmpty()) { if (log.isDebugEnabled()) { log.debug("[{}] Schedule replay of {} messages for {} consumers", name, @@ -1343,15 +1343,20 @@ public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata } } - protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead, long bytesToRead) { + int cappedMaxMessagesToRead = cursor.applyMaxSizeCap(maxMessagesToRead, bytesToRead); + if (cappedMaxMessagesToRead < maxMessagesToRead && log.isDebugEnabled()) { + log.debug("[{}] Capped max messages to read from redelivery list to {} (max was {})", + name, cappedMaxMessagesToRead, maxMessagesToRead); + } if (delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().hasMessageAvailable()) { delayedDeliveryTracker.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis()); NavigableSet messagesAvailableNow = - delayedDeliveryTracker.get().getScheduledMessages(maxMessagesToRead); + delayedDeliveryTracker.get().getScheduledMessages(cappedMaxMessagesToRead); messagesAvailableNow.forEach(p -> redeliveryMessages.add(p.getLedgerId(), p.getEntryId())); } if (!redeliveryMessages.isEmpty()) { - return redeliveryMessages.getMessagesToReplayNow(maxMessagesToRead, createFilterForReplay()); + return redeliveryMessages.getMessagesToReplayNow(cappedMaxMessagesToRead, createFilterForReplay()); } else { return Collections.emptyNavigableSet(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersReadLimitsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersReadLimitsTest.java new file mode 100644 index 0000000000000..a5dc452329836 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersReadLimitsTest.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.commons.lang3.RandomUtils; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.testinterceptor.BrokerTestInterceptor; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.SubscriptionType; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class PersistentDispatcherMultipleConsumersReadLimitsTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + // start at max batch size to reproduce the issue more easily + conf.setDispatcherMinReadBatchSize(conf.getDispatcherMaxReadBatchSize()); + BrokerTestInterceptor.INSTANCE.configure(conf); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @AfterMethod(alwaysRun = true) + protected void resetInterceptors() throws Exception { + BrokerTestInterceptor.INSTANCE.reset(); + } + + @Test(timeOut = 30 * 1000) + public void testDispatcherMaxReadSizeBytes() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName( + "persistent://public/default/testDispatcherMaxReadSizeBytes"); + final String subscription = "sub"; + + AtomicInteger entriesReadMax = new AtomicInteger(0); + BrokerTestInterceptor.INSTANCE.applyDispatcherSpyDecorator(PersistentDispatcherMultipleConsumers.class, + spy -> { + doAnswer(invocation -> { + List entries = invocation.getArgument(0); + PersistentDispatcherMultipleConsumers.ReadType readType = invocation.getArgument(1); + int numberOfEntries = entries.size(); + log.info("intercepted readEntriesComplete with {} entries, read type {}", numberOfEntries, + readType); + entriesReadMax.updateAndGet(current -> Math.max(current, numberOfEntries)); + return invocation.callRealMethod(); + }).when(spy).readEntriesComplete(any(), any()); + } + ); + + // Create two consumers on a shared subscription + @Cleanup + Consumer consumer1 = pulsarClient.newConsumer() + .consumerName("c1") + .topic(topicName) + .subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(10000) + .subscribe(); + + @Cleanup + Consumer consumer2 = pulsarClient.newConsumer() + .consumerName("c2") + .topic(topicName) + .subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared) + .startPaused(true) + .receiverQueueSize(10000) + .subscribe(); + + @Cleanup + Producer producer = + pulsarClient.newProducer().enableBatching(false).topic(topicName).create(); + int numberOfMessages = 200; + int payLoadSizeBytes = 1025 * 1024; // 1025kB + byte[] payload = RandomUtils.nextBytes(payLoadSizeBytes); + for (int i = 0; i < numberOfMessages; i++) { + producer.send(payload); + } + + // Consume messages with consumer1 but don't ack + for (int i = 0; i < numberOfMessages; i++) { + consumer1.receive(); + } + + // Close consumer1 and resume consumer2 to replay the messages + consumer1.close(); + consumer2.resume(); + + // Verify that consumer2 can receive the messages + for (int i = 0; i < numberOfMessages; i++) { + Message msg = consumer2.receive(1, TimeUnit.SECONDS); + Assert.assertNotNull(msg, "Consumer2 should receive the message"); + consumer2.acknowledge(msg); + } + + int expectedMaxEntriesInRead = conf.getDispatcherMaxReadSizeBytes() / payLoadSizeBytes; + assertThat(entriesReadMax.get()).isLessThanOrEqualTo(expectedMaxEntriesInRead); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index 7234f0caefc63..aec161f237be0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -191,6 +191,10 @@ public void setup() throws Exception { doReturn(null).when(cursorMock).getLastIndividualDeletedRange(); doReturn(subscriptionName).when(cursorMock).getName(); doReturn(ledgerMock).when(cursorMock).getManagedLedger(); + doAnswer(invocation -> { + int max = invocation.getArgument(0); + return max; + }).when(cursorMock).applyMaxSizeCap(anyInt(), anyLong()); consumerMock = createMockConsumer(); channelMock = mock(ChannelPromise.class); From 6afd414a29cb7472fec23c2993c0429a057678a2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 27 Jan 2025 11:07:51 +0200 Subject: [PATCH 276/327] [fix][build] Use amazoncorretto:21-alpine3.20 JDK build for Alpine 3.20 (#23898) --- docker/pulsar/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 63fadb62e64ae..121982e44285a 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -55,7 +55,7 @@ RUN chmod -R o+rx /pulsar RUN echo 'OPTS="$OPTS -Dorg.xerial.snappy.use.systemlib=true"' >> /pulsar/conf/bkenv.sh ### Create one stage to include JVM distribution -FROM amazoncorretto:${IMAGE_JDK_MAJOR_VERSION}-alpine AS jvm +FROM amazoncorretto:${IMAGE_JDK_MAJOR_VERSION}-alpine${ALPINE_VERSION} AS jvm RUN apk add --no-cache binutils From 331a997b76b83b3eca777c4559eb60b940d30c27 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 28 Jan 2025 03:26:32 +0800 Subject: [PATCH 277/327] [fix][broker] Fix repeatedly acquired pending reads quota (#23869) --- .../impl/cache/PendingReadsManager.java | 8 +- .../impl/cache/RangeEntryCacheImpl.java | 26 +- .../InflightReadsLimiterIntegrationTest.java | 231 ++++++++++++++++++ .../impl/cache/PendingReadsManagerTest.java | 2 +- 4 files changed, 251 insertions(+), 16 deletions(-) create mode 100644 managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java index 8b2f3e25f1cbb..d733b54dd1304 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java @@ -362,7 +362,7 @@ public void readEntriesFailed(ManagedLedgerException exception, }; rangeEntryCache.asyncReadEntry0(lh, missingOnRight.startEntry, missingOnRight.endEntry, - shouldCacheEntry, readFromRightCallback, null); + shouldCacheEntry, readFromRightCallback, null, false); } @Override @@ -372,7 +372,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object dummyCtx4 } }; rangeEntryCache.asyncReadEntry0(lh, missingOnLeft.startEntry, missingOnLeft.endEntry, - shouldCacheEntry, readFromLeftCallback, null); + shouldCacheEntry, readFromLeftCallback, null, false); } else if (missingOnLeft != null) { AsyncCallbacks.ReadEntriesCallback readFromLeftCallback = new AsyncCallbacks.ReadEntriesCallback() { @@ -395,7 +395,7 @@ public void readEntriesFailed(ManagedLedgerException exception, } }; rangeEntryCache.asyncReadEntry0(lh, missingOnLeft.startEntry, missingOnLeft.endEntry, - shouldCacheEntry, readFromLeftCallback, null); + shouldCacheEntry, readFromLeftCallback, null, false); } else if (missingOnRight != null) { AsyncCallbacks.ReadEntriesCallback readFromRightCallback = new AsyncCallbacks.ReadEntriesCallback() { @@ -418,7 +418,7 @@ public void readEntriesFailed(ManagedLedgerException exception, } }; rangeEntryCache.asyncReadEntry0(lh, missingOnRight.startEntry, missingOnRight.endEntry, - shouldCacheEntry, readFromRightCallback, null); + shouldCacheEntry, readFromRightCallback, null, false); } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java index cb006a5f0cea9..d52fc8535b55b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java @@ -57,7 +57,7 @@ public class RangeEntryCacheImpl implements EntryCache { /** * Overhead per-entry to take into account the envelope. */ - private static final long BOOKKEEPER_READ_OVERHEAD_PER_ENTRY = 64; + public static final long BOOKKEEPER_READ_OVERHEAD_PER_ENTRY = 64; private final RangeEntryCacheManagerImpl manager; final ManagedLedgerImpl ml; @@ -102,7 +102,7 @@ public String getName() { } @VisibleForTesting - InflightReadsLimiter getPendingReadsLimiter() { + public InflightReadsLimiter getPendingReadsLimiter() { return manager.getInflightReadsLimiter(); } @@ -282,7 +282,7 @@ private void asyncReadEntry0(ReadHandle lh, Position position, final ReadEntryCa public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry, final ReadEntriesCallback callback, Object ctx) { try { - asyncReadEntry0(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx); + asyncReadEntry0(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx, true); } catch (Throwable t) { log.warn("failed to read entries for {}--{}-{}", lh.getId(), firstEntry, lastEntry, t); // invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt @@ -295,16 +295,20 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boole @SuppressWarnings({ "unchecked", "rawtypes" }) void asyncReadEntry0(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry, - final ReadEntriesCallback callback, Object ctx) { - asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx, null); + final ReadEntriesCallback callback, Object ctx, boolean withLimits) { + asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx, null, withLimits); } void asyncReadEntry0WithLimits(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry, - final ReadEntriesCallback originalCallback, Object ctx, InflightReadsLimiter.Handle handle) { - - final AsyncCallbacks.ReadEntriesCallback callback = - handlePendingReadsLimits(lh, firstEntry, lastEntry, shouldCacheEntry, - originalCallback, ctx, handle); + final ReadEntriesCallback originalCallback, Object ctx, InflightReadsLimiter.Handle handle, + boolean withLimits) { + AsyncCallbacks.ReadEntriesCallback callback; + if (withLimits) { + callback = handlePendingReadsLimits(lh, firstEntry, lastEntry, shouldCacheEntry, originalCallback, ctx, + handle); + } else { + callback = originalCallback; + } if (callback == null) { return; } @@ -382,7 +386,7 @@ private AsyncCallbacks.ReadEntriesCallback handlePendingReadsLimits(ReadHandle l } ml.getExecutor().execute(() -> { asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry, - originalCallback, ctx, newHandle); + originalCallback, ctx, newHandle, true); }); return null; } else { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java new file mode 100644 index 0000000000000..b57dea6a5bb4d --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; +import org.apache.bookkeeper.mledger.impl.cache.InflightReadsLimiter; +import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; +import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl; +import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.mockito.Mockito; +import org.mockito.stubbing.Answer; +import org.testng.Assert; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +public class InflightReadsLimiterIntegrationTest extends MockedBookKeeperTestCase { + + @DataProvider + public Object[][] readMissingCases() { + return new Object[][]{ + {"missRight"}, + {"missLeft"}, + {"bothMiss"} + }; + } + + @Test(dataProvider = "readMissingCases") + public void testPreciseLimitation(String missingCase) throws Exception { + final long start1 = 50; + final long start2 = "missLeft".endsWith(missingCase) || "bothMiss".equals(missingCase) ? 30 : 50; + final long end1 = 99; + final long end2 = "missRight".endsWith(missingCase) || "bothMiss".equals(missingCase) ? 109 : 99; + final HashSet secondReadEntries = new HashSet<>(); + if (start2 < start1) { + secondReadEntries.add(start2); + } + if (end2 > end1) { + secondReadEntries.add(end1 + 1); + } + final int readCount1 = (int) (end1 - start1 + 1); + final int readCount2 = (int) (end2 - start2 + 1); + + final DefaultThreadFactory threadFactory = new DefaultThreadFactory(UUID.randomUUID().toString()); + final ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(100000); + ManagedLedgerFactoryConfig factoryConfig = new ManagedLedgerFactoryConfig(); + factoryConfig.setCacheEvictionIntervalMs(3600 * 1000); + factoryConfig.setManagedLedgerMaxReadsInFlightSize(1000_000); + final ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConfig); + final ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open("my_test_ledger", config); + final RangeEntryCacheImpl entryCache = (RangeEntryCacheImpl) ml.entryCache; + final RangeEntryCacheManagerImpl rangeEntryCacheManager = + (RangeEntryCacheManagerImpl) factory.getEntryCacheManager(); + final InflightReadsLimiter limiter = rangeEntryCacheManager.getInflightReadsLimiter(); + final long totalCapacity =limiter.getRemainingBytes(); + // final ManagedCursorImpl c1 = (ManagedCursorImpl) ml.openCursor("c1"); + for (byte i = 1; i < 127; i++) { + log.info("add entry: " + i); + ml.addEntry(new byte[]{i}); + } + // Evict cached entries. + entryCache.evictEntries(ml.currentLedgerSize); + Assert.assertEquals(entryCache.getSize(), 0); + + CountDownLatch readCompleteSignal1 = new CountDownLatch(1); + CountDownLatch readCompleteSignal2 = new CountDownLatch(1); + CountDownLatch firstReadingStarted = new CountDownLatch(1); + LedgerHandle currentLedger = ml.currentLedger; + LedgerHandle spyCurrentLedger = Mockito.spy(currentLedger); + ml.currentLedger = spyCurrentLedger; + Answer answer = invocation -> { + long firstEntry = (long) invocation.getArguments()[0]; + log.info("reading entry: {}", firstEntry); + if (firstEntry == start1) { + // Wait 3s to make + firstReadingStarted.countDown(); + readCompleteSignal1.await(); + Object res = invocation.callRealMethod(); + return res; + } else if(secondReadEntries.contains(firstEntry)) { + final CompletableFuture res = new CompletableFuture<>(); + threadFactory.newThread(() -> { + try { + readCompleteSignal2.await(); + CompletableFuture future = + (CompletableFuture) invocation.callRealMethod(); + future.thenAccept(v -> { + res.complete(v); + }).exceptionally(ex -> { + res.completeExceptionally(ex); + return null; + }); + } catch (Throwable ex) { + res.completeExceptionally(ex); + } + }).start(); + return res; + } else { + return invocation.callRealMethod(); + } + }; + doAnswer(answer).when(spyCurrentLedger).readAsync(anyLong(), anyLong()); + doAnswer(answer).when(spyCurrentLedger).readUnconfirmedAsync(anyLong(), anyLong()); + + // Initialize "entryCache.estimatedEntrySize" to the correct value. + Object ctx = new Object(); + SimpleReadEntriesCallback cb0 = new SimpleReadEntriesCallback(); + entryCache.asyncReadEntry(spyCurrentLedger, 125, 125, true, cb0, ctx); + cb0.entries.join(); + Long sizePerEntry1 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); + Assert.assertEquals(sizePerEntry1, 1); + Awaitility.await().untilAsserted(() -> { + long remainingBytes =limiter.getRemainingBytes(); + Assert.assertEquals(remainingBytes, totalCapacity); + }); + log.info("remainingBytes 0: {}", limiter.getRemainingBytes()); + + // Concurrency reading. + + SimpleReadEntriesCallback cb1 = new SimpleReadEntriesCallback(); + SimpleReadEntriesCallback cb2 = new SimpleReadEntriesCallback(); + threadFactory.newThread(() -> { + entryCache.asyncReadEntry(spyCurrentLedger, start1, end1, true, cb1, ctx); + }).start(); + threadFactory.newThread(() -> { + try { + firstReadingStarted.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + entryCache.asyncReadEntry(spyCurrentLedger, start2, end2, true, cb2, ctx); + }).start(); + + long bytesAcquired1 = calculateBytesSizeBeforeFirstReading(readCount1 + readCount2, 1); + long remainingBytesExpected1 = totalCapacity - bytesAcquired1; + log.info("acquired : {}", bytesAcquired1); + log.info("remainingBytesExpected 0 : {}", remainingBytesExpected1); + Awaitility.await().untilAsserted(() -> { + log.info("remainingBytes 0: {}", limiter.getRemainingBytes()); + Assert.assertEquals(limiter.getRemainingBytes(), remainingBytesExpected1); + }); + + // Complete the read1. + Thread.sleep(3000); + readCompleteSignal1.countDown(); + cb1.entries.join(); + Long sizePerEntry2 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); + Assert.assertEquals(sizePerEntry2, 1); + long bytesAcquired2 = calculateBytesSizeBeforeFirstReading(readCount2, 1); + long remainingBytesExpected2 = totalCapacity - bytesAcquired2; + log.info("acquired : {}", bytesAcquired2); + log.info("remainingBytesExpected 1: {}", remainingBytesExpected2); + Awaitility.await().untilAsserted(() -> { + log.info("remainingBytes 1: {}", limiter.getRemainingBytes()); + Assert.assertEquals(limiter.getRemainingBytes(), remainingBytesExpected2); + }); + + readCompleteSignal2.countDown(); + cb2.entries.join(); + Long sizePerEntry3 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); + Assert.assertEquals(sizePerEntry3, 1); + Awaitility.await().untilAsserted(() -> { + long remainingBytes = limiter.getRemainingBytes(); + log.info("remainingBytes 2: {}", remainingBytes); + Assert.assertEquals(remainingBytes, totalCapacity); + }); + // cleanup + ml.delete(); + factory.shutdown(); + } + + private long calculateBytesSizeBeforeFirstReading(int entriesCount, int perEntrySize) { + return entriesCount * (perEntrySize + RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY); + } + + class SimpleReadEntriesCallback implements AsyncCallbacks.ReadEntriesCallback { + + CompletableFuture> entries = new CompletableFuture<>(); + + @Override + public void readEntriesComplete(List entriesRead, Object ctx) { + List list = new ArrayList<>(entriesRead.size()); + for (Entry entry : entriesRead) { + byte b = entry.getDataBuffer().readByte(); + list.add(b); + entry.release(); + } + this.entries.complete(list); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + this.entries.completeExceptionally(exception); + } + } +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java index 01976f648aba4..383568c17e83d 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java @@ -108,7 +108,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(rangeEntryCache).asyncReadEntry0(any(), anyLong(), anyLong(), - anyBoolean(), any(), any()); + anyBoolean(), any(), any(), anyBoolean()); lh = mock(ReadHandle.class); ml = mock(ManagedLedgerImpl.class); From 08270dd99f7c166316175c8f437b976f9dd69e44 Mon Sep 17 00:00:00 2001 From: Lee hong <40360529+walkinggo@users.noreply.github.com> Date: Wed, 29 Jan 2025 00:03:11 +0800 Subject: [PATCH 278/327] [fix][fn] Record Pulsar Function processing time properly for asynchronous functions (#23811) Co-authored-by: Zixuan Liu --- .../pulsar/io/PulsarFunctionE2ETest.java | 95 +++++++++++++++++++ .../instance/JavaExecutionResult.java | 6 +- .../functions/instance/JavaInstance.java | 12 +-- .../instance/JavaInstanceRunnable.java | 12 ++- .../instance/stats/ComponentStatsManager.java | 3 +- .../instance/stats/FunctionStatsManager.java | 11 +-- .../instance/stats/SinkStatsManager.java | 6 +- .../instance/stats/SourceStatsManager.java | 6 +- .../src/main/resources/findbugsExclude.xml | 5 + .../instance/JavaInstanceRunnableTest.java | 23 +++++ 10 files changed, 142 insertions(+), 37 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java index 74c2a93b84e9f..aef75e5fc7efb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java @@ -64,6 +64,7 @@ import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.compaction.PublishingOrderCompactor; import org.apache.pulsar.functions.api.Context; +import org.apache.pulsar.functions.api.examples.JavaNativeAsyncExclamationFunction; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.worker.FunctionRuntimeManager; @@ -296,6 +297,100 @@ public void testReadCompactedFunction() throws Exception { producer.close(); } + @Test(timeOut = 20000) + public void testPulsarFunctionAsyncStatTime() throws Exception { + final String namespacePortion = "io"; + final String replNamespace = tenant + "/" + namespacePortion; + final String sourceTopic = "persistent://" + replNamespace + "/my-topic1"; + final String sinkTopic = "persistent://" + replNamespace + "/output"; + final String functionName = "JavaNativeAsyncExclamationFunction"; + final String subscriptionName = "test-sub"; + admin.namespaces().createNamespace(replNamespace); + Set clusters = Sets.newHashSet(Lists.newArrayList("use")); + admin.namespaces().setNamespaceReplicationClusters(replNamespace, clusters); + + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespacePortion); + functionConfig.setName(functionName); + functionConfig.setParallelism(1); + functionConfig.setSubName(subscriptionName); + functionConfig.setInputSpecs(Collections.singletonMap(sourceTopic, + ConsumerConfig.builder().poolMessages(true).build())); + functionConfig.setAutoAck(true); + functionConfig.setClassName(JavaNativeAsyncExclamationFunction.class.getName()); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + functionConfig.setOutput(sinkTopic); + functionConfig.setCleanupSubscription(true); + functionConfig.setProcessingGuarantees(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE); + + admin.functions().createFunctionWithUrl(functionConfig, + PulsarFunctionE2ETest.class.getProtectionDomain().getCodeSource().getLocation().toURI().toString()); + + // create a producer that creates a topic at broker + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(sourceTopic).create(); + Consumer consumer = + pulsarClient.newConsumer(Schema.STRING).topic(sinkTopic).subscriptionName(subscriptionName).subscribe(); + + retryStrategically((test) -> { + try { + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 1; + } catch (PulsarAdminException e) { + return false; + } + }, 50, 150); + retryStrategically((test) -> { + try { + return admin.topics().getStats(sinkTopic).getSubscriptions().size() == 1; + } catch (PulsarAdminException e) { + return false; + } + }, 50, 150); + // validate pulsar sink consumer has started on the topic + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 1); + assertEquals(admin.topics().getStats(sinkTopic).getSubscriptions().size(), 1); + + int cntMsg = 5; + for (int i = 0; i < cntMsg; i++) { + producer.newMessage().value("it is the " + i + "th message , it will spend 500ms").send(); + } + Awaitility.await().ignoreExceptions().untilAsserted(() -> { + SubscriptionStats subStats = admin.topics().getStats(sourceTopic).getSubscriptions().get(subscriptionName); + assertEquals(subStats.getUnackedMessages(), 0); + }); + int count = 0; + while (true) { + Message message = consumer.receive(10, TimeUnit.SECONDS); + if (message == null) { + break; + } + consumer.acknowledge(message); + count++; + } + Assert.assertEquals(count, cntMsg); + + String prometheusMetrics = TestPulsarFunctionUtils.getPrometheusMetrics(pulsar.getListenPortHTTP().get()); + log.info("prometheus metrics: {}", prometheusMetrics); + Map statsMetrics = + TestPulsarFunctionUtils.parseMetrics(prometheusMetrics); + + assertEquals(statsMetrics.get("pulsar_function_process_latency_ms").value, 500.0, 100.0); + admin.functions().deleteFunction(tenant, namespacePortion, functionName); + + retryStrategically((test) -> { + try { + return admin.topics().getStats(sourceTopic).getSubscriptions().size() == 0; + } catch (PulsarAdminException e) { + return false; + } + }, 50, 150); + + // make sure subscriptions are cleanup + assertEquals(admin.topics().getStats(sourceTopic).getSubscriptions().size(), 0); + + tempDirectory.assertThatFunctionDownloadTempFilesHaveBeenDeleted(); + } + @Test(timeOut = 20000) public void testPulsarFunctionStats() throws Exception { diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaExecutionResult.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaExecutionResult.java index 5856600196b49..9ca9aa2a879d4 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaExecutionResult.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaExecutionResult.java @@ -29,9 +29,5 @@ public class JavaExecutionResult { private Throwable userException; private Object result; - - public void reset() { - setUserException(null); - setResult(null); - } + private final long startTime = System.nanoTime(); } diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java index 5946be9fe5be9..c5f82898f8251 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java @@ -47,6 +47,7 @@ public class JavaInstance implements AutoCloseable { public static class AsyncFuncRequest { private final Record record; private final CompletableFuture processResult; + private final JavaExecutionResult result; } @Getter(AccessLevel.PACKAGE) @@ -136,7 +137,7 @@ public JavaExecutionResult handleMessage(Record record, Object input, if (asyncPreserveInputOrderForOutputMessages) { // Function is in format: Function> AsyncFuncRequest request = new AsyncFuncRequest( - record, (CompletableFuture) output + record, (CompletableFuture) output, executionResult ); pendingAsyncRequests.put(request); } else { @@ -148,13 +149,12 @@ public JavaExecutionResult handleMessage(Record record, Object input, processAsyncResultsInInputOrder(asyncResultConsumer); } else { try { - JavaExecutionResult execResult = new JavaExecutionResult(); if (cause != null) { - execResult.setUserException(FutureUtil.unwrapCompletionException(cause)); + executionResult.setUserException(FutureUtil.unwrapCompletionException(cause)); } else { - execResult.setResult(res); + executionResult.setResult(res); } - asyncResultConsumer.accept(record, execResult); + asyncResultConsumer.accept(record, executionResult); } finally { asyncRequestsConcurrencyLimiter.release(); } @@ -187,7 +187,7 @@ private void processAsyncResultsInInputOrder(JavaInstanceRunnable.AsyncResultCon while (asyncResult != null && asyncResult.getProcessResult().isDone()) { pendingAsyncRequests.remove(asyncResult); - JavaExecutionResult execResult = new JavaExecutionResult(); + JavaExecutionResult execResult = asyncResult.getResult(); try { Object result = asyncResult.getProcessResult().get(); execResult.setResult(result); diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java index 4f811c14704a0..cfb7e9536a3e6 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java @@ -334,8 +334,6 @@ public void run() { // set last invocation time stats.setLastInvocation(System.currentTimeMillis()); - // start time for process latency stat - stats.processTimeStart(); // process the message Thread.currentThread().setContextClassLoader(functionClassLoader); @@ -346,9 +344,6 @@ public void run() { asyncErrorHandler); Thread.currentThread().setContextClassLoader(instanceClassLoader); - // register end time - stats.processTimeEnd(); - if (result != null) { // process the synchronous results handleResult(currentRecord, result); @@ -448,6 +443,8 @@ void handleResult(Record srcRecord, JavaExecutionResult result) throws Exception // increment total successfully processed stats.incrTotalProcessedSuccessfully(); } + // handle endTime here + stats.processTimeEnd(result.getStartTime()); } private void sendOutputMessage(Record srcRecord, Object output) throws Exception { @@ -631,6 +628,11 @@ public String getStatsAsString() throws IOException { return ""; } + @VisibleForTesting + void setStats(ComponentStatsManager stats) { + this.stats = stats; + } + public InstanceCommunication.MetricsData getAndResetMetrics() { if (isInitialized) { statsLock.writeLock().lock(); diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/ComponentStatsManager.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/ComponentStatsManager.java index 6da3c082f78f4..17321735256eb 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/ComponentStatsManager.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/ComponentStatsManager.java @@ -100,9 +100,8 @@ public ComponentStatsManager(FunctionCollectorRegistry collectorRegistry, public abstract void setLastInvocation(long ts); - public abstract void processTimeStart(); - public abstract void processTimeEnd(); + public abstract void processTimeEnd(long startTime); public abstract double getTotalProcessedSuccessfully(); diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/FunctionStatsManager.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/FunctionStatsManager.java index 8737c8a4fa913..0009fcea6671a 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/FunctionStatsManager.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/FunctionStatsManager.java @@ -336,20 +336,13 @@ public void setLastInvocation(long ts) { statlastInvocationChild.set(ts); } - private Long processTimeStart; - @Override - public void processTimeStart() { - processTimeStart = System.nanoTime(); - } @Override - public void processTimeEnd() { - if (processTimeStart != null) { - double endTimeMs = ((double) System.nanoTime() - processTimeStart) / 1.0E6D; + public void processTimeEnd(long startTime) { + double endTimeMs = ((double) System.nanoTime() - startTime) / 1.0E6D; statProcessLatencyChild.observe(endTimeMs); statProcessLatency1minChild.observe(endTimeMs); - } } @Override diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SinkStatsManager.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SinkStatsManager.java index c515ce6bc872c..4fae7f9c292d1 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SinkStatsManager.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SinkStatsManager.java @@ -279,13 +279,9 @@ public void setLastInvocation(long ts) { statlastInvocationChild.set(ts); } - @Override - public void processTimeStart() { - //no-op - } @Override - public void processTimeEnd() { + public void processTimeEnd(long startTime) { //no-op } diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SourceStatsManager.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SourceStatsManager.java index 1f7e159c4dcb5..b68e1d610f7d3 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SourceStatsManager.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/stats/SourceStatsManager.java @@ -279,13 +279,9 @@ public void setLastInvocation(long ts) { statlastInvocationChild.set(ts); } - @Override - public void processTimeStart() { - //no-op - } @Override - public void processTimeEnd() { + public void processTimeEnd(long startTime) { //no-op } diff --git a/pulsar-functions/instance/src/main/resources/findbugsExclude.xml b/pulsar-functions/instance/src/main/resources/findbugsExclude.xml index 40e3e91112328..ffe23993eb702 100644 --- a/pulsar-functions/instance/src/main/resources/findbugsExclude.xml +++ b/pulsar-functions/instance/src/main/resources/findbugsExclude.xml @@ -557,4 +557,9 @@ + + + + + diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceRunnableTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceRunnableTest.java index c83648132d488..385d78e671727 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceRunnableTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceRunnableTest.java @@ -24,7 +24,9 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; + import com.fasterxml.jackson.annotation.JsonIgnore; + import java.lang.reflect.Field; import java.lang.reflect.Method; import java.time.Duration; @@ -36,6 +38,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicReference; + import lombok.Getter; import lombok.Setter; import lombok.extern.slf4j.Slf4j; @@ -50,6 +53,7 @@ import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.api.SerDe; import org.apache.pulsar.functions.instance.stats.ComponentStatsManager; +import org.apache.pulsar.functions.instance.stats.FunctionStatsManager; import org.apache.pulsar.functions.proto.Function.FunctionDetails; import org.apache.pulsar.functions.proto.Function.SinkSpec; import org.apache.pulsar.functions.proto.Function.SourceSpec; @@ -61,6 +65,7 @@ import org.apache.pulsar.io.core.SourceContext; import org.awaitility.Awaitility; import org.jetbrains.annotations.NotNull; +import org.mockito.ArgumentCaptor; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; @@ -173,6 +178,24 @@ public Void process(String input, Context context) throws Exception { } } + @Test + public void testFunctionAsyncTime() throws Exception { + FunctionDetails functionDetails = FunctionDetails.newBuilder() + .setAutoAck(true) + .setProcessingGuarantees(org.apache.pulsar.functions.proto.Function.ProcessingGuarantees.MANUAL) + .build(); + JavaInstanceRunnable javaInstanceRunnable = createRunnable(functionDetails); + FunctionStatsManager manager = mock(FunctionStatsManager.class); + javaInstanceRunnable.setStats(manager); + JavaExecutionResult javaExecutionResult = new JavaExecutionResult(); + Thread.sleep(500); + Record record = mock(Record.class); + javaInstanceRunnable.handleResult(record, javaExecutionResult); + ArgumentCaptor timeCaptor = ArgumentCaptor.forClass(Long.class); + verify(manager).processTimeEnd(timeCaptor.capture()); + Assert.assertEquals(timeCaptor.getValue(), javaExecutionResult.getStartTime()); + } + @Test public void testFunctionResultNull() throws Exception { JavaExecutionResult javaExecutionResult = new JavaExecutionResult(); From 9079262aaedce719bda786a63781292a4257e299 Mon Sep 17 00:00:00 2001 From: Philipp Dolif <52791955+pdolif@users.noreply.github.com> Date: Tue, 28 Jan 2025 23:17:21 +0100 Subject: [PATCH 279/327] [fix][test] Fix flaky DelayedDeliveryTest.testEnableTopicDelayedDelivery (#23893) --- .../pulsar/broker/service/persistent/DelayedDeliveryTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index e47857e8ec60f..793a1767fd8c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -483,10 +483,10 @@ public void testEnableTopicDelayedDelivery() throws Exception { break; } } - producer.newMessage().value("long-tick-msg").deliverAfter(2, TimeUnit.SECONDS).send(); + producer.newMessage().value("long-tick-msg").deliverAfter(3, TimeUnit.SECONDS).send(); msg = consumer.receive(1, TimeUnit.SECONDS); assertNull(msg); - msg = consumer.receive(3, TimeUnit.SECONDS); + msg = consumer.receive(4, TimeUnit.SECONDS); assertNotNull(msg); } From b6cfecce5f3a1eecbf6f5df81cb835fbbfe35980 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 29 Jan 2025 01:23:52 +0200 Subject: [PATCH 280/327] [fix][broker Fix bug in RangeCache where different instance of the key wouldn't ever match (#23903) --- .../bookkeeper/mledger/util/RangeCache.java | 109 +++++-- .../mledger/impl/ManagedLedgerBkTest.java | 115 ++++++- .../mledger/util/RangeCacheTest.java | 19 +- .../KeySharedSubscriptionBrokerCacheTest.java | 308 ++++++++++++++++++ pulsar-broker/src/test/resources/log4j2.xml | 9 + .../impl/TxnLogBufferedWriterTest.java | 9 + .../test/MockedBookKeeperTestCase.java | 6 +- 7 files changed, 543 insertions(+), 32 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionBrokerCacheTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java index 2f2b161a30684..0de6f94362215 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java @@ -106,7 +106,39 @@ K getKey() { return localKey; } + /** + * Get the value associated with the key. Returns null if the key does not match the key. + * + * @param key the key to match + * @return the value associated with the key, or null if the value has already been recycled or the key does not + * match + */ V getValue(K key) { + return getValueInternal(key, false); + } + + /** + * Get the value associated with the Map.Entry's key and value. Exact instance of the key is required to match. + * @param entry the entry which contains the key and {@link EntryWrapper} value to get the value from + * @return the value associated with the key, or null if the value has already been recycled or the key does not + * exactly match the same instance + */ + static V getValueMatchingMapEntry(Map.Entry> entry) { + return entry.getValue().getValueInternal(entry.getKey(), true); + } + + /** + * Get the value associated with the key. Returns null if the key does not match the key associated with the + * value. + * + * @param key the key to match + * @param requireSameKeyInstance when true, the matching will be restricted to exactly the same instance of the + * key as the one stored in the wrapper. This is used to avoid any races + * when retrieving or removing the entries from the cache when the key and value + * instances are available. + * @return the value associated with the key, or null if the key does not match + */ + private V getValueInternal(K key, boolean requireSameKeyInstance) { long stamp = lock.tryOptimisticRead(); K localKey = this.key; V localValue = this.value; @@ -116,7 +148,11 @@ V getValue(K key) { localValue = this.value; lock.unlockRead(stamp); } - if (localKey != key) { + // check that the given key matches the key associated with the value in the entry + // this is used to detect if the entry has already been recycled and contains another key + // when requireSameKeyInstance is true, the key must be exactly the same instance as the one stored in the + // entry to match + if (localKey != key && (requireSameKeyInstance || localKey == null || !localKey.equals(key))) { return null; } return localValue; @@ -236,34 +272,45 @@ public boolean exists(Key key) { * The caller is responsible for releasing the reference. */ public Value get(Key key) { - return getValue(key, entries.get(key)); + return getValueFromWrapper(key, entries.get(key)); } - private Value getValue(Key key, EntryWrapper valueWrapper) { + private Value getValueFromWrapper(Key key, EntryWrapper valueWrapper) { if (valueWrapper == null) { return null; } else { Value value = valueWrapper.getValue(key); - if (value == null) { - // the wrapper has been recycled and contains another key - return null; - } - try { - value.retain(); - } catch (IllegalReferenceCountException e) { - // Value was already deallocated - return null; - } - // check that the value matches the key and that there's at least 2 references to it since - // the cache should be holding one reference and a new reference was just added in this method - if (value.refCnt() > 1 && value.matchesKey(key)) { - return value; - } else { - // Value or IdentityWrapper was recycled and already contains another value - // release the reference added in this method - value.release(); - return null; - } + return getRetainedValueMatchingKey(key, value); + } + } + + private Value getValueMatchingEntry(Map.Entry> entry) { + Value valueMatchingEntry = EntryWrapper.getValueMatchingMapEntry(entry); + return getRetainedValueMatchingKey(entry.getKey(), valueMatchingEntry); + } + + // validates that the value matches the key and that the value has not been recycled + // which are possible due to the lack of exclusive locks in the cache and the use of reference counted objects + private Value getRetainedValueMatchingKey(Key key, Value value) { + if (value == null) { + // the wrapper has been recycled and contains another key + return null; + } + try { + value.retain(); + } catch (IllegalReferenceCountException e) { + // Value was already deallocated + return null; + } + // check that the value matches the key and that there's at least 2 references to it since + // the cache should be holding one reference and a new reference was just added in this method + if (value.refCnt() > 1 && value.matchesKey(key)) { + return value; + } else { + // Value or IdentityWrapper was recycled and already contains another value + // release the reference added in this method + value.release(); + return null; } } @@ -280,7 +327,7 @@ public Collection getRange(Key first, Key last) { // Return the values of the entries found in cache for (Map.Entry> entry : entries.subMap(first, true, last, true).entrySet()) { - Value value = getValue(entry.getKey(), entry.getValue()); + Value value = getValueMatchingEntry(entry); if (value != null) { values.add(value); } @@ -297,6 +344,9 @@ public Collection getRange(Key first, Key last) { * @return an pair of ints, containing the number of removed entries and the total size */ public Pair removeRange(Key first, Key last, boolean lastInclusive) { + if (log.isDebugEnabled()) { + log.debug("Removing entries in range [{}, {}], lastInclusive: {}", first, last, lastInclusive); + } RemovalCounters counters = RemovalCounters.create(); Map> subMap = entries.subMap(first, true, last, lastInclusive); for (Map.Entry> entry : subMap.entrySet()) { @@ -320,7 +370,7 @@ private RemoveEntryResult removeEntry(Map.Entry> e boolean skipInvalid, Predicate removeCondition) { Key key = entry.getKey(); EntryWrapper entryWrapper = entry.getValue(); - Value value = entryWrapper.getValue(key); + Value value = getValueMatchingEntry(entry); if (value == null) { // the wrapper has already been recycled and contains another key if (!skipInvalid) { @@ -404,6 +454,9 @@ private Pair handleRemovalResult(RemovalCounters counters) { * @return a pair containing the number of entries evicted and their total size */ public Pair evictLeastAccessedEntries(long minSize) { + if (log.isDebugEnabled()) { + log.debug("Evicting entries to reach a minimum size of {}", minSize); + } checkArgument(minSize > 0); RemovalCounters counters = RemovalCounters.create(); while (counters.removedSize < minSize && !Thread.currentThread().isInterrupted()) { @@ -422,6 +475,9 @@ public Pair evictLeastAccessedEntries(long minSize) { * @return the tota */ public Pair evictLEntriesBeforeTimestamp(long maxTimestamp) { + if (log.isDebugEnabled()) { + log.debug("Evicting entries with timestamp <= {}", maxTimestamp); + } RemovalCounters counters = RemovalCounters.create(); while (!Thread.currentThread().isInterrupted()) { Map.Entry> entry = entries.firstEntry(); @@ -453,6 +509,9 @@ public long getSize() { * @return size of removed entries */ public Pair clear() { + if (log.isDebugEnabled()) { + log.debug("Clearing the cache with {} entries and size {}", entries.size(), size.get()); + } RemovalCounters counters = RemovalCounters.create(); while (!Thread.currentThread().isInterrupted()) { Map.Entry> entry = entries.firstEntry(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java index e23937afea2c9..574ed2f325136 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java @@ -24,24 +24,28 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.buffer.ByteBuf; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; - +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperTestClient; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; import org.apache.bookkeeper.mledger.Entry; @@ -53,18 +57,17 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.mledger.util.ThrowableToStringUtil; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; +import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import io.netty.buffer.ByteBuf; -import lombok.Cleanup; - @Slf4j public class ManagedLedgerBkTest extends BookKeeperClusterTestCase { @@ -241,6 +244,108 @@ public void verifyConcurrentUsage() throws Exception { assertEquals(factory.getMbean().getNumberOfCacheEvictions(), 0); } + @Test + public void verifyAsyncReadEntryUsingCache() throws Exception { + ManagedLedgerFactoryConfig config = new ManagedLedgerFactoryConfig(); + + config.setMaxCacheSize(100 * 1024 * 1024); + config.setCacheEvictionTimeThresholdMillis(10000); + config.setCacheEvictionIntervalMs(10000); + + @Cleanup("shutdown") + ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, config); + + ManagedLedgerConfig conf = new ManagedLedgerConfig(); + conf.setEnsembleSize(2).setAckQuorumSize(2).setMetadataEnsembleSize(2) + .setRetentionSizeInMB(-1).setRetentionTime(-1, TimeUnit.MILLISECONDS); + final ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my-ledger" + testName, conf); + + int NumProducers = 5; + int NumConsumers = 10; + + final AtomicBoolean done = new AtomicBoolean(); + final CyclicBarrier barrier = new CyclicBarrier(NumProducers + NumConsumers + 1); + + List> futures = new ArrayList(); + List positions = new CopyOnWriteArrayList<>(); + + for (int i = 0; i < NumProducers; i++) { + futures.add(executor.submit(() -> { + try { + // wait for all threads to be ready to start at once + barrier.await(); + while (!done.get()) { + Position position = ledger.addEntry("entry".getBytes()); + positions.add(position); + Thread.sleep(1); + } + } catch (Exception e) { + e.printStackTrace(); + throw FutureUtil.wrapToCompletionException(e); + } + })); + } + + // create a dummy cursor since caching happens only when there are active consumers + ManagedCursor cursor = ledger.openCursor("dummy"); + + for (int i = 0; i < NumConsumers; i++) { + futures.add(executor.submit(() -> { + try { + // wait for all threads to be ready to start at once + barrier.await(); + while (!done.get()) { + if (positions.isEmpty()) { + Thread.sleep(1); + continue; + } + // Simulate a replay queue read pattern where individual entries are read + Position randomPosition = positions.get(ThreadLocalRandom.current().nextInt(positions.size())); + // Clone the original instance so that another instance is used in the asyncReadEntry call + // This is to test that keys are compared by .equals and not by reference under the covers + randomPosition = PositionFactory.create(randomPosition); + CompletableFuture future = new CompletableFuture<>(); + ledger.asyncReadEntry(randomPosition, new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + entry.release(); + future.complete(null); + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null); + future.get(); + Thread.sleep(2); + } + } catch (Exception e) { + e.printStackTrace(); + throw FutureUtil.wrapToCompletionException(e); + } + })); + } + + // trigger all worker threads at once to continue from the barrier + barrier.await(); + + int testDurationSeconds = 3; + Thread.sleep(testDurationSeconds * 1000); + + done.set(true); + for (Future future : futures) { + future.get(); + } + + factory.getMbean().refreshStats(testDurationSeconds, TimeUnit.SECONDS); + + assertTrue(factory.getMbean().getCacheHitsRate() > 0.0); + assertEquals(factory.getMbean().getCacheMissesRate(), 0.0); + assertTrue(factory.getMbean().getCacheHitsThroughput() > 0.0); + assertEquals(factory.getMbean().getNumberOfCacheEvictions(), 0); + } + @Test public void testSimple() throws Exception { @Cleanup("shutdown") diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java index 4bcf2cc6c4e35..aa13d4b8e3488 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java @@ -20,6 +20,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotSame; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -338,4 +339,20 @@ public void testRemoveEntryWithInvalidMatchingKey() { cache.clear(); assertEquals(cache.getNumberOfEntries(), 0); } -} + + @Test + public void testGetKeyWithDifferentInstance() { + RangeCache cache = new RangeCache<>(); + Integer key = 129; + cache.put(key, new RefString("129")); + // create a different instance of the key + Integer key2 = Integer.valueOf(129); + // key and key2 are different instances but they are equal + assertNotSame(key, key2); + assertEquals(key, key2); + // get the value using key2 + RefString s = cache.get(key2); + // the value should be found + assertEquals(s.s, "129"); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionBrokerCacheTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionBrokerCacheTest.java new file mode 100644 index 0000000000000..9fca95e2e8719 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionBrokerCacheTest.java @@ -0,0 +1,308 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; +import static org.assertj.core.api.SoftAssertions.assertSoftly; +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import lombok.Cleanup; +import lombok.SneakyThrows; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.tests.KeySharedImplementationType; +import org.awaitility.Awaitility; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; +import org.testng.annotations.Test; + +@Test(groups = "broker-impl") +public class KeySharedSubscriptionBrokerCacheTest extends ProducerConsumerBase { + private static final Logger log = LoggerFactory.getLogger(KeySharedSubscriptionBrokerCacheTest.class); + private static final String SUBSCRIPTION_NAME = "key_shared"; + private final KeySharedImplementationType implementationType; + + // Comment out the next line (Factory annotation) to run tests manually in IntelliJ, one-by-one + @Factory + public static Object[] createTestInstances() { + return KeySharedImplementationType.generateTestInstances(KeySharedSubscriptionBrokerCacheTest::new); + } + + public KeySharedSubscriptionBrokerCacheTest() { + // set the default implementation type for manual running in IntelliJ + this(KeySharedImplementationType.PIP379); + } + + public KeySharedSubscriptionBrokerCacheTest(KeySharedImplementationType implementationType) { + this.implementationType = implementationType; + } + + @DataProvider(name = "currentImplementationType") + public Object[] currentImplementationType() { + return new Object[]{ implementationType }; + } + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(implementationType.classic); + conf.setSubscriptionSharedUseClassicPersistentImplementation(implementationType.classic); + conf.setUnblockStuckSubscriptionEnabled(false); + conf.setSubscriptionKeySharedUseConsistentHashing(true); + conf.setManagedLedgerCacheSizeMB(100); + + // configure to evict entries after 30 seconds so that we can test retrieval from cache + conf.setManagedLedgerCacheEvictionTimeThresholdMillis(30000); + conf.setManagedLedgerCacheEvictionIntervalMs(30000); + + // Important: this is currently necessary to make use of cache for replay queue reads + conf.setCacheEvictionByMarkDeletedPosition(true); + + conf.setManagedLedgerMaxReadsInFlightSizeInMB(100); + conf.setDispatcherRetryBackoffInitialTimeInMs(0); + conf.setDispatcherRetryBackoffMaxTimeInMs(0); + conf.setKeySharedUnblockingIntervalMs(0); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @AfterMethod(alwaysRun = true) + public void resetAfterMethod() throws Exception { + List list = admin.namespaces().getTopics("public/default"); + for (String topicName : list){ + if (!pulsar.getBrokerService().isSystemTopic(topicName)) { + admin.topics().delete(topicName, false); + } + } + pulsarTestContext.getMockBookKeeper().setReadHandleInterceptor(null); + } + + // Use a fixed seed to make the tests using random values deterministic + // When a test fails, it's possible to re-run it to reproduce the issue + private static final Random random = new Random(1); + + private Producer createProducer(String topic, boolean enableBatch) throws PulsarClientException { + Producer producer = null; + if (enableBatch) { + producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .enableBatching(true) + .maxPendingMessages(2001) + .batcherBuilder(BatcherBuilder.KEY_BASED) + .create(); + } else { + producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .maxPendingMessages(2001) + .enableBatching(false) + .create(); + } + return producer; + } + + private StickyKeyConsumerSelector getSelector(String topic, String subscription) { + return getStickyKeyDispatcher(topic, subscription).getSelector(); + } + + @SneakyThrows + private StickyKeyDispatcher getStickyKeyDispatcher(String topic, String subscription) { + Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); + StickyKeyDispatcher dispatcher = (StickyKeyDispatcher) sub.getDispatcher(); + return dispatcher; + } + + @Test(dataProvider = "currentImplementationType", invocationCount = 1) + public void testReplayQueueReadsGettingCached(KeySharedImplementationType impl) throws Exception { + String topic = newUniqueName("testReplayQueueReadsGettingCached"); + int numberOfKeys = 100; + long pauseTime = 100L; + + @Cleanup + Producer producer = createProducer(topic, false); + + // create a consumer and close it to create a subscription + pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe() + .close(); + + Set remainingMessageValues = Collections.synchronizedSet(new HashSet<>()); + BlockingQueue, Message>> unackedMessages = new LinkedBlockingQueue<>(); + AtomicBoolean c2MessagesShouldBeUnacked = new AtomicBoolean(true); + Set keysForC2 = new HashSet<>(); + AtomicLong lastMessageTimestamp = new AtomicLong(System.currentTimeMillis()); + + MessageListener messageHandler = (consumer, msg) -> { + lastMessageTimestamp.set(System.currentTimeMillis()); + synchronized (this) { + String key = msg.getKey(); + if (c2MessagesShouldBeUnacked.get() && keysForC2.contains(key)) { + unackedMessages.add(Pair.of(consumer, msg)); + return; + } + remainingMessageValues.remove(msg.getValue()); + consumer.acknowledgeAsync(msg); + } + }; + + pulsarTestContext.getMockBookKeeper().setReadHandleInterceptor((ledgerId, firstEntry, lastEntry, entries) -> { + log.error("Attempting to read from BK when cache should be used. {}:{} to {}:{}", ledgerId, firstEntry, + ledgerId, lastEntry); + return CompletableFuture.failedFuture( + new ManagedLedgerException.NonRecoverableLedgerException( + "Should not read from BK since cache should be used.")); + }); + + // Adding a new consumer. + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .messageListener(messageHandler) + .subscribe(); + + @Cleanup + Consumer c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .messageListener(messageHandler) + .subscribe(); + + @Cleanup + Consumer c3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .messageListener(messageHandler) + .subscribe(); + + StickyKeyDispatcher dispatcher = getStickyKeyDispatcher(topic, SUBSCRIPTION_NAME); + StickyKeyConsumerSelector selector = dispatcher.getSelector(); + + // find keys that will be assigned to c2 + for (int i = 0; i < numberOfKeys; i++) { + String key = String.valueOf(i); + byte[] keyBytes = key.getBytes(UTF_8); + int hash = selector.makeStickyKeyHash(keyBytes); + if (selector.select(hash).consumerName().equals("c2")) { + keysForC2.add(key); + } + } + + // close c2 + c2.close(); + + // produce messages with random keys + for (int i = 0; i < 1000; i++) { + String key = String.valueOf(random.nextInt(numberOfKeys)); + //log.info("Producing message with key: {} value: {}", key, i); + remainingMessageValues.add(i); + producer.newMessage() + .key(key) + .value(i) + .send(); + } + + // reconnect c2 + c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .messageListener(messageHandler) + .startPaused(true) + .subscribe(); + + // ack the unacked messages to unblock c2 keys + c2MessagesShouldBeUnacked.set(false); + Pair, Message> consumerMessagePair; + while ((consumerMessagePair = unackedMessages.poll()) != null) { + messageHandler.received(consumerMessagePair.getLeft(), consumerMessagePair.getRight()); + } + + // produce more messages with random keys + for (int i = 0; i < 1000; i++) { + String key = String.valueOf(random.nextInt(numberOfKeys)); + //log.info("Producing message with key: {} value: {}", key, i); + remainingMessageValues.add(i); + producer.newMessage() + .key(key) + .value(i) + .send(); + } + + c2.resume(); + + Awaitility.await().atMost(Duration.ofSeconds(10)).until(() -> { + return remainingMessageValues.isEmpty() + || System.currentTimeMillis() - lastMessageTimestamp.get() > 50 * pauseTime; + }); + + try { + assertSoftly(softly -> { + softly.assertThat(remainingMessageValues).as("remainingMessageValues").isEmpty(); + ManagedLedgerFactoryMXBean cacheStats = pulsar.getDefaultManagedLedgerFactory().getCacheStats(); + softly.assertThat(cacheStats.getCacheHitsTotal()).as("cache hits").isGreaterThan(0); + softly.assertThat(cacheStats.getCacheMissesTotal()).as("cache misses").isEqualTo(0); + softly.assertThat(cacheStats.getNumberOfCacheEvictions()).as("cache evictions").isEqualTo(0); + }); + } finally { + logTopicStats(topic); + } + } +} diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml index 7b3cd6a04fcca..b348a1d04b797 100644 --- a/pulsar-broker/src/test/resources/log4j2.xml +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -53,5 +53,14 @@ --> + + diff --git a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java index 3147279477843..f7c343d7421f3 100644 --- a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java +++ b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java @@ -55,6 +55,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; @@ -127,6 +128,14 @@ public Object[][] mainProcessCasesProvider(){ }; } + @Override + protected ManagedLedgerFactoryConfig createManagedLedgerFactoryConfig() { + ManagedLedgerFactoryConfig managedLedgerFactoryConfig = super.createManagedLedgerFactoryConfig(); + // disable the broker cache so that assertAllByteBufHasBeenReleased can work correctly. + managedLedgerFactoryConfig.setMaxCacheSize(0); + return managedLedgerFactoryConfig; + } + /** * Tests all operations from write to callback, including these step: * 1. Write many data. diff --git a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/test/MockedBookKeeperTestCase.java b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/test/MockedBookKeeperTestCase.java index ac5aa3bd8927e..e3e6945620c34 100644 --- a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/test/MockedBookKeeperTestCase.java +++ b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/test/MockedBookKeeperTestCase.java @@ -82,10 +82,14 @@ public void setUp(Method method) throws Exception { throw e; } - ManagedLedgerFactoryConfig conf = new ManagedLedgerFactoryConfig(); + ManagedLedgerFactoryConfig conf = createManagedLedgerFactoryConfig(); factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, conf); } + protected ManagedLedgerFactoryConfig createManagedLedgerFactoryConfig() { + return new ManagedLedgerFactoryConfig(); + } + @AfterMethod(alwaysRun = true) public void tearDown(Method method) { try { From ed5dbb5289f09f913f07a38ba1481727f00f063a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 29 Jan 2025 19:33:40 +0200 Subject: [PATCH 281/327] [fix][ci] Configure Docker data-root to /mnt/docker to avoid running out of disk space (#23909) --- .github/actions/clean-disk/action.yml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/.github/actions/clean-disk/action.yml b/.github/actions/clean-disk/action.yml index d74c3f25fc64c..e67ce59a08ddb 100644 --- a/.github/actions/clean-disk/action.yml +++ b/.github/actions/clean-disk/action.yml @@ -46,6 +46,15 @@ runs: time df -BM / /mnt echo "::endgroup::" done + if [[ "${{ inputs.mode }}" == "full" ]]; then + echo "::group::Moving /var/lib/docker to /mnt/docker" + sudo systemctl stop docker + echo '{"data-root": "/mnt/docker"}' | sudo tee /etc/docker/daemon.json + sudo mv /var/lib/docker /mnt/docker + sudo systemctl start docker + time df -BM / /mnt + echo "::endgroup::" + fi echo "::group::Cleaning apt state" time sudo bash -c "apt-get clean; apt-get autoclean; apt-get -y --purge autoremove" time df -BM / /mnt From c5173d5e15efade90afb9b0b1c19f3ba5b3aab37 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 29 Jan 2025 22:11:56 +0200 Subject: [PATCH 282/327] [fix][broker] Make InflightReadsLimiter asynchronous and apply it for replay queue reads (#23901) --- .../mledger/ManagedLedgerFactoryConfig.java | 12 + .../impl/ManagedLedgerFactoryImpl.java | 2 +- .../impl/cache/InflightReadsLimiter.java | 252 ++++++-- .../impl/cache/PendingReadsManager.java | 156 +++-- .../impl/cache/RangeEntryCacheImpl.java | 298 +++++----- .../cache/RangeEntryCacheManagerImpl.java | 16 +- .../InflightReadsLimiterIntegrationTest.java | 13 +- .../mledger/impl/ManagedLedgerTest.java | 92 +-- .../impl/cache/InflightReadsLimiterTest.java | 560 ++++++++++++++---- .../impl/cache/PendingReadsManagerTest.java | 4 +- .../pulsar/broker/ServiceConfiguration.java | 9 + .../broker/ManagedLedgerClientFactory.java | 17 +- 12 files changed, 984 insertions(+), 447 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java index 386310b3ccbae..af538262ed44a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java @@ -61,6 +61,18 @@ public class ManagedLedgerFactoryConfig { */ private long managedLedgerMaxReadsInFlightSize = 0; + /** + * Maximum time to wait for acquiring permits for max reads in flight when managedLedgerMaxReadsInFlightSizeInMB is + * set (>0) and the limit is reached. + */ + private long managedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis = 60000; + + /** + * Maximum number of reads that can be queued for acquiring permits for max reads in flight when + * managedLedgerMaxReadsInFlightSizeInMB is set (>0) and the limit is reached. + */ + private int managedLedgerMaxReadsInFlightPermitsAcquireQueueSize = 10000; + /** * Whether trace managed ledger task execution time. */ diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 12c3ea12df581..225f4dba493d5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -228,7 +228,7 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, compressionConfigForManagedCursorInfo); this.config = config; this.mbean = new ManagedLedgerFactoryMBeanImpl(this); - this.entryCacheManager = new RangeEntryCacheManagerImpl(this, openTelemetry); + this.entryCacheManager = new RangeEntryCacheManagerImpl(this, scheduledExecutor, openTelemetry); this.statsTask = scheduledExecutor.scheduleWithFixedDelay(catchingAndLoggingThrowables(this::refreshStats), 0, config.getStatsPeriodSeconds(), TimeUnit.SECONDS); this.flushCursorsTask = scheduledExecutor.scheduleAtFixedRate(catchingAndLoggingThrowables(this::flushCursors), diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java index c87807b86631b..1f4d2c267975c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java @@ -22,12 +22,16 @@ import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.metrics.ObservableLongCounter; import io.prometheus.client.Gauge; -import lombok.AllArgsConstructor; -import lombok.ToString; +import java.util.Optional; +import java.util.Queue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.opentelemetry.Constants; import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization; import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; +import org.jctools.queues.SpscArrayQueue; @Slf4j public class InflightReadsLimiter implements AutoCloseable { @@ -58,16 +62,36 @@ public class InflightReadsLimiter implements AutoCloseable { private final long maxReadsInFlightSize; private long remainingBytes; + private final long acquireTimeoutMillis; + private final ScheduledExecutorService timeOutExecutor; + private final boolean enabled; - public InflightReadsLimiter(long maxReadsInFlightSize, OpenTelemetry openTelemetry) { - if (maxReadsInFlightSize <= 0) { + record Handle(long permits, long creationTime, boolean success) { + } + + record QueuedHandle(Handle handle, Consumer callback) { + } + + private final Queue queuedHandles; + private boolean timeoutCheckRunning = false; + + public InflightReadsLimiter(long maxReadsInFlightSize, int maxReadsInFlightAcquireQueueSize, + long acquireTimeoutMillis, ScheduledExecutorService timeOutExecutor, + OpenTelemetry openTelemetry) { + this.maxReadsInFlightSize = maxReadsInFlightSize; + this.remainingBytes = maxReadsInFlightSize; + this.acquireTimeoutMillis = acquireTimeoutMillis; + this.timeOutExecutor = timeOutExecutor; + if (maxReadsInFlightSize > 0) { + enabled = true; + this.queuedHandles = new SpscArrayQueue<>(maxReadsInFlightAcquireQueueSize); + } else { + enabled = false; + this.queuedHandles = null; // set it to -1 in order to show in the metrics that the metric is not available PULSAR_ML_READS_BUFFER_SIZE.set(-1); PULSAR_ML_READS_AVAILABLE_BUFFER_SIZE.set(-1); } - this.maxReadsInFlightSize = maxReadsInFlightSize; - this.remainingBytes = maxReadsInFlightSize; - var meter = openTelemetry.getMeter(Constants.BROKER_INSTRUMENTATION_SCOPE_NAME); inflightReadsLimitCounter = meter.counterBuilder(INFLIGHT_READS_LIMITER_LIMIT_METRIC_NAME) .setDescription("Maximum number of bytes that can be retained by managed ledger data read from storage " @@ -102,70 +126,178 @@ public void close() { inflightReadsUsageCounter.close(); } - @AllArgsConstructor - @ToString - static class Handle { - final long acquiredPermits; - final boolean success; - final int trials; + private static final Handle DISABLED = new Handle(0, 0, true); + private static final Optional DISABLED_OPTIONAL = Optional.of(DISABLED); - final long creationTime; + /** + * Acquires permits from the limiter. If the limiter is disabled, it will immediately return a successful handle. + * If permits are available, it will return a handle with the acquired permits. If no permits are available, + * it will return an empty optional and the callback will be called when permits become available or when the + * acquire timeout is reached. The success field in the handle passed to the callback will be false if the acquire + * operation times out. The callback should be non-blocking and run on a desired executor handled within the + * callback itself. + * + * A successful handle will have the success field set to true, and the caller must call release with the handle + * when the permits are no longer needed. + * + * If an unsuccessful handle is returned immediately, it means that the queue limit has been reached and the + * callback will not be called. The caller should fail the read operation in this case to apply backpressure. + * + * @param permits the number of permits to acquire + * @param callback the callback to be called when the permits are acquired or timed out + * @return an optional handle that contains the permits if acquired, otherwise an empty optional + */ + public Optional acquire(long permits, Consumer callback) { + if (isDisabled()) { + return DISABLED_OPTIONAL; + } + return internalAcquire(permits, callback); } - private static final Handle DISABLED = new Handle(0, true, 0, -1); + private synchronized Optional internalAcquire(long permits, Consumer callback) { + Handle handle = new Handle(permits, System.currentTimeMillis(), true); + if (remainingBytes >= permits) { + remainingBytes -= permits; + if (log.isDebugEnabled()) { + log.debug("acquired permits: {}, creationTime: {}, remainingBytes:{}", permits, handle.creationTime, + remainingBytes); + } + updateMetrics(); + return Optional.of(handle); + } else if (permits > maxReadsInFlightSize && remainingBytes == maxReadsInFlightSize) { + remainingBytes = 0; + if (log.isInfoEnabled()) { + log.info("Requested permits {} exceeded maxReadsInFlightSize {}, creationTime: {}, remainingBytes:{}. " + + "Allowing request with permits set to maxReadsInFlightSize.", + permits, maxReadsInFlightSize, handle.creationTime, remainingBytes); + } + updateMetrics(); + return Optional.of(new Handle(maxReadsInFlightSize, handle.creationTime, true)); + } else { + if (queuedHandles.offer(new QueuedHandle(handle, callback))) { + scheduleTimeOutCheck(acquireTimeoutMillis); + return Optional.empty(); + } else { + log.warn("Failed to queue handle for acquiring permits: {}, creationTime: {}, remainingBytes:{}", + permits, handle.creationTime, remainingBytes); + return Optional.of(new Handle(0, handle.creationTime, false)); + } + } + } - Handle acquire(long permits, Handle current) { - if (maxReadsInFlightSize <= 0) { - // feature is disabled - return DISABLED; + private synchronized void scheduleTimeOutCheck(long delayMillis) { + if (acquireTimeoutMillis <= 0) { + return; } - synchronized (this) { - try { - if (current == null) { - if (remainingBytes == 0) { - return new Handle(0, false, 1, System.currentTimeMillis()); - } - if (remainingBytes >= permits) { - remainingBytes -= permits; - return new Handle(permits, true, 1, System.currentTimeMillis()); - } else { - long possible = remainingBytes; - remainingBytes = 0; - return new Handle(possible, false, 1, System.currentTimeMillis()); - } + if (!timeoutCheckRunning) { + timeoutCheckRunning = true; + timeOutExecutor.schedule(this::timeoutCheck, delayMillis, TimeUnit.MILLISECONDS); + } + } + + private synchronized void timeoutCheck() { + timeoutCheckRunning = false; + long delay = 0; + while (true) { + QueuedHandle queuedHandle = queuedHandles.peek(); + if (queuedHandle != null) { + long age = System.currentTimeMillis() - queuedHandle.handle.creationTime; + if (age >= acquireTimeoutMillis) { + // remove the peeked handle from the queue + queuedHandles.poll(); + handleTimeout(queuedHandle); } else { - if (current.trials >= 4 && current.acquiredPermits > 0) { - remainingBytes += current.acquiredPermits; - return new Handle(0, false, 1, current.creationTime); - } - if (remainingBytes == 0) { - return new Handle(current.acquiredPermits, false, current.trials + 1, - current.creationTime); - } - long needed = permits - current.acquiredPermits; - if (remainingBytes >= needed) { - remainingBytes -= needed; - return new Handle(permits, true, current.trials + 1, current.creationTime); - } else { - long possible = remainingBytes; - remainingBytes = 0; - return new Handle(current.acquiredPermits + possible, false, - current.trials + 1, current.creationTime); - } + delay = acquireTimeoutMillis - age; + break; } - } finally { - updateMetrics(); + } else { + break; } } + if (delay > 0) { + scheduleTimeOutCheck(delay); + } + } + + private void handleTimeout(QueuedHandle queuedHandle) { + if (log.isDebugEnabled()) { + log.debug("timed out queued permits: {}, creationTime: {}, remainingBytes:{}", + queuedHandle.handle.permits, queuedHandle.handle.creationTime, remainingBytes); + } + try { + queuedHandle.callback.accept(new Handle(0, queuedHandle.handle.creationTime, false)); + } catch (Exception e) { + log.error("Error in callback of timed out queued permits: {}, creationTime: {}, remainingBytes:{}", + queuedHandle.handle.permits, queuedHandle.handle.creationTime, remainingBytes, e); + } } - void release(Handle handle) { + /** + * Releases permits back to the limiter. If the handle is disabled, this method will be a no-op. + * + * @param handle the handle containing the permits to release + */ + public void release(Handle handle) { if (handle == DISABLED) { return; } - synchronized (this) { - remainingBytes += handle.acquiredPermits; - updateMetrics(); + internalRelease(handle); + } + + private synchronized void internalRelease(Handle handle) { + if (log.isDebugEnabled()) { + log.debug("release permits: {}, creationTime: {}, remainingBytes:{}", handle.permits, + handle.creationTime, getRemainingBytes()); + } + remainingBytes += handle.permits; + while (true) { + QueuedHandle queuedHandle = queuedHandles.peek(); + if (queuedHandle != null) { + boolean timedOut = acquireTimeoutMillis > 0 + && System.currentTimeMillis() - queuedHandle.handle.creationTime > acquireTimeoutMillis; + if (timedOut) { + // remove the peeked handle from the queue + queuedHandles.poll(); + handleTimeout(queuedHandle); + } else if (remainingBytes >= queuedHandle.handle.permits + || queuedHandle.handle.permits > maxReadsInFlightSize + && remainingBytes == maxReadsInFlightSize) { + // remove the peeked handle from the queue + queuedHandles.poll(); + handleQueuedHandle(queuedHandle); + } else { + break; + } + } else { + break; + } + } + updateMetrics(); + } + + private void handleQueuedHandle(QueuedHandle queuedHandle) { + long permits = queuedHandle.handle.permits; + Handle handleForCallback = queuedHandle.handle; + if (permits > maxReadsInFlightSize && remainingBytes == maxReadsInFlightSize) { + remainingBytes = 0; + if (log.isInfoEnabled()) { + log.info("Requested permits {} exceeded maxReadsInFlightSize {}, creationTime: {}, remainingBytes:{}. " + + "Allowing request with permits set to maxReadsInFlightSize.", + permits, maxReadsInFlightSize, queuedHandle.handle.creationTime, remainingBytes); + } + handleForCallback = new Handle(maxReadsInFlightSize, queuedHandle.handle.creationTime, true); + } else { + remainingBytes -= permits; + if (log.isDebugEnabled()) { + log.debug("acquired queued permits: {}, creationTime: {}, remainingBytes:{}", + permits, queuedHandle.handle.creationTime, remainingBytes); + } + } + try { + queuedHandle.callback.accept(handleForCallback); + } catch (Exception e) { + log.error("Error in callback of acquired queued permits: {}, creationTime: {}, remainingBytes:{}", + handleForCallback.permits, handleForCallback.creationTime, remainingBytes, e); } } @@ -175,8 +307,6 @@ private synchronized void updateMetrics() { } public boolean isDisabled() { - return maxReadsInFlightSize <= 0; + return !enabled; } - - -} +} \ No newline at end of file diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java index d733b54dd1304..5944199287ee1 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java @@ -25,9 +25,8 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; -import lombok.AllArgsConstructor; -import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.mledger.AsyncCallbacks; @@ -95,15 +94,11 @@ public PendingReadsManager(RangeEntryCacheImpl rangeEntryCache) { this.rangeEntryCache = rangeEntryCache; } - @Value - private static class PendingReadKey { - private final long startEntry; - private final long endEntry; + private record PendingReadKey(long startEntry, long endEntry) { long size() { return endEntry - startEntry + 1; } - boolean includes(PendingReadKey other) { return startEntry <= other.startEntry && other.endEntry <= endEntry; } @@ -135,25 +130,18 @@ PendingReadKey reminderOnRight(PendingReadKey other) { } - @AllArgsConstructor - private static final class ReadEntriesCallbackWithContext { - final AsyncCallbacks.ReadEntriesCallback callback; - final Object ctx; - final long startEntry; - final long endEntry; + private record ReadEntriesCallbackWithContext(AsyncCallbacks.ReadEntriesCallback callback, Object ctx, + long startEntry, long endEntry) { } - @AllArgsConstructor - private static final class FindPendingReadOutcome { - final PendingRead pendingRead; - final PendingReadKey missingOnLeft; - final PendingReadKey missingOnRight; + private record FindPendingReadOutcome(PendingRead pendingRead, + PendingReadKey missingOnLeft, PendingReadKey missingOnRight) { boolean needsAdditionalReads() { return missingOnLeft != null || missingOnRight != null; } } - private FindPendingReadOutcome findPendingRead(PendingReadKey key, Map ledgerCache, AtomicBoolean created) { synchronized (ledgerCache) { PendingRead existing = ledgerCache.get(key); @@ -222,18 +210,74 @@ private FindPendingReadOutcome findPendingRead(PendingReadKey key, Map ledgerCache; + final ConcurrentMap ledgerCache; final List callbacks = new ArrayList<>(1); boolean completed = false; public PendingRead(PendingReadKey key, - Map ledgerCache) { + ConcurrentMap ledgerCache) { this.key = key; this.ledgerCache = ledgerCache; } - private List keepEntries(List list, long startEntry, long endEntry) { - List result = new ArrayList<>((int) (endEntry - startEntry)); + public void attach(CompletableFuture> handle) { + handle.whenComplete((entriesToReturn, error) -> { + // execute in the completing thread + completeAndRemoveFromCache(); + // execute the callbacks in the managed ledger executor + rangeEntryCache.getManagedLedger().getExecutor().execute(() -> { + if (error != null) { + readEntriesFailed(error); + } else { + readEntriesComplete(entriesToReturn); + } + }); + }); + } + + private synchronized void completeAndRemoveFromCache() { + completed = true; + // When the read has completed, remove the instance from the ledgerCache map + // so that new reads will go to a new instance. + // this is required because we are going to do refcount management + // on the results of the callback + ledgerCache.remove(key, this); + } + + private synchronized void readEntriesComplete(List entriesToReturn) { + if (callbacks.size() == 1) { + ReadEntriesCallbackWithContext first = callbacks.get(0); + if (first.startEntry == key.startEntry + && first.endEntry == key.endEntry) { + // perfect match, no copy, this is the most common case + first.callback.readEntriesComplete((List) entriesToReturn, + first.ctx); + } else { + first.callback.readEntriesComplete( + keepEntries(entriesToReturn, first.startEntry, first.endEntry), + first.ctx); + } + } else { + for (ReadEntriesCallbackWithContext callback : callbacks) { + callback.callback.readEntriesComplete( + copyEntries(entriesToReturn, callback.startEntry, callback.endEntry), + callback.ctx); + } + for (EntryImpl entry : entriesToReturn) { + entry.release(); + } + } + } + + private synchronized void readEntriesFailed(Throwable error) { + for (ReadEntriesCallbackWithContext callback : callbacks) { + ManagedLedgerException mlException = createManagedLedgerException(error); + callback.callback.readEntriesFailed(mlException, callback.ctx); + } + } + + private List keepEntries(List list, long startEntry, long endEntry) { + List result = new ArrayList<>((int) (endEntry - startEntry)); for (EntryImpl entry : list) { long entryId = entry.getEntryId(); if (startEntry <= entryId && entryId <= endEntry) { @@ -245,62 +289,16 @@ private List keepEntries(List list, long startEntry, long return result; } - public void attach(CompletableFuture> handle) { - // when the future is done remove this from the map - // new reads will go to a new instance - // this is required because we are going to do refcount management - // on the results of the callback - handle.whenComplete((___, error) -> { - synchronized (PendingRead.this) { - completed = true; - synchronized (ledgerCache) { - ledgerCache.remove(key, this); - } - } - }); - - handle.thenAcceptAsync(entriesToReturn -> { - synchronized (PendingRead.this) { - if (callbacks.size() == 1) { - ReadEntriesCallbackWithContext first = callbacks.get(0); - if (first.startEntry == key.startEntry - && first.endEntry == key.endEntry) { - // perfect match, no copy, this is the most common case - first.callback.readEntriesComplete((List) entriesToReturn, - first.ctx); - } else { - first.callback.readEntriesComplete( - (List) keepEntries(entriesToReturn, first.startEntry, first.endEntry), - first.ctx); - } - } else { - for (ReadEntriesCallbackWithContext callback : callbacks) { - long callbackStartEntry = callback.startEntry; - long callbackEndEntry = callback.endEntry; - List copy = new ArrayList<>((int) (callbackEndEntry - callbackStartEntry + 1)); - for (EntryImpl entry : entriesToReturn) { - long entryId = entry.getEntryId(); - if (callbackStartEntry <= entryId && entryId <= callbackEndEntry) { - EntryImpl entryCopy = EntryImpl.create(entry); - copy.add(entryCopy); - } - } - callback.callback.readEntriesComplete((List) copy, callback.ctx); - } - for (EntryImpl entry : entriesToReturn) { - entry.release(); - } - } - } - }, rangeEntryCache.getManagedLedger().getExecutor()).exceptionally(exception -> { - synchronized (PendingRead.this) { - for (ReadEntriesCallbackWithContext callback : callbacks) { - ManagedLedgerException mlException = createManagedLedgerException(exception); - callback.callback.readEntriesFailed(mlException, callback.ctx); - } + private List copyEntries(List entriesToReturn, long startEntry, long endEntry) { + List result = new ArrayList<>((int) (endEntry - startEntry + 1)); + for (EntryImpl entry : entriesToReturn) { + long entryId = entry.getEntryId(); + if (startEntry <= entryId && entryId <= endEntry) { + EntryImpl entryCopy = EntryImpl.create(entry); + result.add(entryCopy); } - return null; - }); + } + return result; } synchronized boolean addListener(AsyncCallbacks.ReadEntriesCallback callback, @@ -318,7 +316,7 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldC final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { final PendingReadKey key = new PendingReadKey(firstEntry, lastEntry); - Map pendingReadsForLedger = + ConcurrentMap pendingReadsForLedger = cachedPendingReads.computeIfAbsent(lh.getId(), (l) -> new ConcurrentHashMap<>()); boolean listenerAdded = false; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java index d52fc8535b55b..b81015ea63988 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java @@ -22,18 +22,19 @@ import static java.util.Objects.requireNonNull; import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; import io.netty.buffer.PooledByteBufAllocator; +import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; +import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.LongAdder; import org.apache.bookkeeper.client.api.BKException; import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.client.api.ReadHandle; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; import org.apache.bookkeeper.mledger.Entry; @@ -58,6 +59,8 @@ public class RangeEntryCacheImpl implements EntryCache { * Overhead per-entry to take into account the envelope. */ public static final long BOOKKEEPER_READ_OVERHEAD_PER_ENTRY = 64; + private static final int DEFAULT_ESTIMATED_ENTRY_SIZE = 10 * 1024; + private static final boolean DEFAULT_CACHE_INDIVIDUAL_READ_ENTRY = false; private final RangeEntryCacheManagerImpl manager; final ManagedLedgerImpl ml; @@ -66,18 +69,16 @@ public class RangeEntryCacheImpl implements EntryCache { private final boolean copyEntries; private final PendingReadsManager pendingReadsManager; - private volatile long estimatedEntrySize = 10 * 1024; - - private final long readEntryTimeoutMillis; - private static final double MB = 1024 * 1024; + private final LongAdder totalAddedEntriesSize = new LongAdder(); + private final LongAdder totalAddedEntriesCount = new LongAdder(); + public RangeEntryCacheImpl(RangeEntryCacheManagerImpl manager, ManagedLedgerImpl ml, boolean copyEntries) { this.manager = manager; this.ml = ml; this.pendingReadsManager = new PendingReadsManager(this); this.interceptor = ml.getManagedLedgerInterceptor(); - this.readEntryTimeoutMillis = getManagedLedgerConfig().getReadEntryTimeoutSeconds(); this.entries = new RangeCache<>(EntryImpl::getLength, EntryImpl::getTimestamp); this.copyEntries = copyEntries; @@ -118,17 +119,18 @@ public InflightReadsLimiter getPendingReadsLimiter() { @Override public boolean insert(EntryImpl entry) { + int entryLength = entry.getLength(); if (!manager.hasSpaceInCache()) { if (log.isDebugEnabled()) { log.debug("[{}] Skipping cache while doing eviction: {} - size: {}", ml.getName(), entry.getPosition(), - entry.getLength()); + entryLength); } return false; } if (log.isDebugEnabled()) { log.debug("[{}] Adding entry to cache: {} - size: {}", ml.getName(), entry.getPosition(), - entry.getLength()); + entryLength); } Position position = entry.getPosition(); @@ -150,7 +152,9 @@ public boolean insert(EntryImpl entry) { EntryImpl cacheEntry = EntryImpl.create(position, cachedData); cachedData.release(); if (entries.put(position, cacheEntry)) { - manager.entryAdded(entry.getLength()); + totalAddedEntriesSize.add(entryLength); + totalAddedEntriesCount.increment(); + manager.entryAdded(entryLength); return true; } else { // entry was not inserted into cache, we need to discard it @@ -226,7 +230,23 @@ public void invalidateAllEntries(long ledgerId) { public void asyncReadEntry(ReadHandle lh, Position position, final ReadEntryCallback callback, final Object ctx) { try { - asyncReadEntry0(lh, position, callback, ctx); + asyncReadEntriesByPosition(lh, position, position, 1, + DEFAULT_CACHE_INDIVIDUAL_READ_ENTRY, + new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + if (entries.isEmpty()) { + callback.readEntryFailed(new ManagedLedgerException("Could not read given position"), ctx); + } else { + callback.readEntryComplete(entries.get(0), ctx); + } + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + callback.readEntryFailed(exception, ctx); + } + }, ctx, true); } catch (Throwable t) { log.warn("failed to read entries for {}-{}", lh.getId(), position, t); // invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt @@ -237,47 +257,6 @@ public void asyncReadEntry(ReadHandle lh, Position position, final ReadEntryCall } } - private void asyncReadEntry0(ReadHandle lh, Position position, final ReadEntryCallback callback, - final Object ctx) { - if (log.isDebugEnabled()) { - log.debug("[{}] Reading entry ledger {}: {}", ml.getName(), lh.getId(), position.getEntryId()); - } - EntryImpl entry = entries.get(position); - if (entry != null) { - EntryImpl cachedEntry = EntryImpl.create(entry); - entry.release(); - manager.mlFactoryMBean.recordCacheHit(cachedEntry.getLength()); - callback.readEntryComplete(cachedEntry, ctx); - } else { - ReadEntryUtils.readAsync(ml, lh, position.getEntryId(), position.getEntryId()).thenAcceptAsync( - ledgerEntries -> { - try { - Iterator iterator = ledgerEntries.iterator(); - if (iterator.hasNext()) { - LedgerEntry ledgerEntry = iterator.next(); - EntryImpl returnEntry = RangeEntryCacheManagerImpl.create(ledgerEntry, interceptor); - - ml.getMbean().recordReadEntriesOpsCacheMisses(1, returnEntry.getLength()); - manager.mlFactoryMBean.recordCacheMiss(1, returnEntry.getLength()); - ml.getMbean().addReadEntriesSample(1, returnEntry.getLength()); - callback.readEntryComplete(returnEntry, ctx); - } else { - // got an empty sequence - callback.readEntryFailed(new ManagedLedgerException("Could not read given position"), - ctx); - } - } finally { - ledgerEntries.close(); - } - }, ml.getExecutor()).exceptionally(exception -> { - ml.invalidateLedgerHandle(lh); - pendingReadsManager.invalidateLedger(lh.getId()); - callback.readEntryFailed(createManagedLedgerException(exception), ctx); - return null; - }); - } - } - @Override public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry, final ReadEntriesCallback callback, Object ctx) { @@ -295,38 +274,123 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boole @SuppressWarnings({ "unchecked", "rawtypes" }) void asyncReadEntry0(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry, - final ReadEntriesCallback callback, Object ctx, boolean withLimits) { - asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry, callback, ctx, null, withLimits); + final ReadEntriesCallback callback, Object ctx, boolean acquirePermits) { + final long ledgerId = lh.getId(); + final int numberOfEntries = (int) (lastEntry - firstEntry) + 1; + final Position firstPosition = PositionFactory.create(ledgerId, firstEntry); + final Position lastPosition = PositionFactory.create(ledgerId, lastEntry); + asyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, shouldCacheEntry, callback, ctx, + acquirePermits); } - void asyncReadEntry0WithLimits(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry, - final ReadEntriesCallback originalCallback, Object ctx, InflightReadsLimiter.Handle handle, - boolean withLimits) { - AsyncCallbacks.ReadEntriesCallback callback; - if (withLimits) { - callback = handlePendingReadsLimits(lh, firstEntry, lastEntry, shouldCacheEntry, originalCallback, ctx, - handle); + void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, + boolean shouldCacheEntry, final ReadEntriesCallback originalCallback, + Object ctx, boolean acquirePermits) { + checkArgument(firstPosition.getLedgerId() == lastPosition.getLedgerId(), + "Invalid range. Entries %s and %s should be in the same ledger.", + firstPosition, lastPosition); + checkArgument(firstPosition.getLedgerId() == lh.getId(), + "Invalid ReadHandle. The ledger %s of the range positions should match the handle's ledger %s.", + firstPosition.getLedgerId(), lh.getId()); + + if (log.isDebugEnabled()) { + log.debug("[{}] Reading {} entries in range {} to {}", ml.getName(), numberOfEntries, firstPosition, + lastPosition); + } + + InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); + if (!acquirePermits || pendingReadsLimiter.isDisabled()) { + doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, shouldCacheEntry, + originalCallback, ctx); } else { - callback = originalCallback; + long estimatedEntrySize = getEstimatedEntrySize(); + long estimatedReadSize = numberOfEntries * estimatedEntrySize; + if (log.isDebugEnabled()) { + log.debug("Estimated read size: {} bytes for {} entries with {} estimated entry size", + estimatedReadSize, + numberOfEntries, estimatedEntrySize); + } + Optional optionalHandle = + pendingReadsLimiter.acquire(estimatedReadSize, handle -> { + // permits were not immediately available, callback will be executed when permits are acquired + // or timeout + ml.getExecutor().execute(() -> { + doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries, + shouldCacheEntry, originalCallback, ctx, handle, estimatedReadSize); + }); + }); + // permits were immediately available and acquired + if (optionalHandle.isPresent()) { + doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries, + shouldCacheEntry, originalCallback, ctx, optionalHandle.get(), estimatedReadSize); + } } - if (callback == null) { + } + + void doAsyncReadEntriesWithAcquiredPermits(ReadHandle lh, Position firstPosition, Position lastPosition, + int numberOfEntries, boolean shouldCacheEntry, + final ReadEntriesCallback originalCallback, Object ctx, + InflightReadsLimiter.Handle handle, long estimatedReadSize) { + if (!handle.success()) { + String message = String.format( + "Couldn't acquire enough permits on the max reads in flight limiter to read from ledger " + + "%d, %s, estimated read size %d bytes for %d entries (check " + + "managedLedgerMaxReadsInFlightSizeInMB, " + + "managedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis and " + + "managedLedgerMaxReadsInFlightPermitsAcquireQueueSize)", lh.getId(), getName(), + estimatedReadSize, numberOfEntries); + log.error(message); + originalCallback.readEntriesFailed(new ManagedLedgerException.TooManyRequestsException(message), ctx); return; } + InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); + ReadEntriesCallback wrappedCallback = new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx2) { + if (!entries.isEmpty()) { + // release permits only when entries have been handled + AtomicInteger remainingCount = new AtomicInteger(entries.size()); + for (Entry entry : entries) { + ((EntryImpl) entry).onDeallocate(() -> { + if (remainingCount.decrementAndGet() <= 0) { + pendingReadsLimiter.release(handle); + } + }); + } + } else { + pendingReadsLimiter.release(handle); + } + originalCallback.readEntriesComplete(entries, ctx2); + } - final long ledgerId = lh.getId(); - final int entriesToRead = (int) (lastEntry - firstEntry) + 1; - final Position firstPosition = PositionFactory.create(lh.getId(), firstEntry); - final Position lastPosition = PositionFactory.create(lh.getId(), lastEntry); + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx2) { + pendingReadsLimiter.release(handle); + originalCallback.readEntriesFailed(exception, ctx2); + } + }; + doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, shouldCacheEntry, + wrappedCallback, ctx); + } - if (log.isDebugEnabled()) { - log.debug("[{}] Reading entries range ledger {}: {} to {}", ml.getName(), ledgerId, firstEntry, lastEntry); + void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, + boolean shouldCacheEntry, final ReadEntriesCallback callback, + Object ctx) { + Collection cachedEntries; + if (firstPosition.compareTo(lastPosition) == 0) { + EntryImpl cachedEntry = entries.get(firstPosition); + if (cachedEntry == null) { + cachedEntries = Collections.emptyList(); + } else { + cachedEntries = Collections.singleton(cachedEntry); + } + } else { + cachedEntries = entries.getRange(firstPosition, lastPosition); } - Collection cachedEntries = entries.getRange(firstPosition, lastPosition); - - if (cachedEntries.size() == entriesToRead) { + if (cachedEntries.size() == numberOfEntries) { long totalCachedSize = 0; - final List entriesToReturn = Lists.newArrayListWithExpectedSize(entriesToRead); + final List entriesToReturn = new ArrayList<>(numberOfEntries); // All entries found in cache for (EntryImpl entry : cachedEntries) { @@ -337,11 +401,11 @@ void asyncReadEntry0WithLimits(ReadHandle lh, long firstEntry, long lastEntry, b manager.mlFactoryMBean.recordCacheHits(entriesToReturn.size(), totalCachedSize); if (log.isDebugEnabled()) { - log.debug("[{}] Ledger {} -- Found in cache entries: {}-{}", ml.getName(), ledgerId, firstEntry, - lastEntry); + log.debug("[{}] Cache hit for {} entries in range {} to {}", ml.getName(), numberOfEntries, + firstPosition, lastPosition); } - callback.readEntriesComplete((List) entriesToReturn, ctx); + callback.readEntriesComplete(entriesToReturn, ctx); } else { if (!cachedEntries.isEmpty()) { @@ -349,77 +413,24 @@ void asyncReadEntry0WithLimits(ReadHandle lh, long firstEntry, long lastEntry, b } // Read all the entries from bookkeeper - pendingReadsManager.readEntries(lh, firstEntry, lastEntry, + pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(), shouldCacheEntry, callback, ctx); - } } - private AsyncCallbacks.ReadEntriesCallback handlePendingReadsLimits(ReadHandle lh, - long firstEntry, long lastEntry, - boolean shouldCacheEntry, - AsyncCallbacks.ReadEntriesCallback originalCallback, - Object ctx, InflightReadsLimiter.Handle handle) { - InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); - if (pendingReadsLimiter.isDisabled()) { - return originalCallback; + @VisibleForTesting + public long getEstimatedEntrySize() { + long estimatedEntrySize = getAvgEntrySize(); + if (estimatedEntrySize == 0) { + estimatedEntrySize = DEFAULT_ESTIMATED_ENTRY_SIZE; } - long estimatedReadSize = (1 + lastEntry - firstEntry) - * (estimatedEntrySize + BOOKKEEPER_READ_OVERHEAD_PER_ENTRY); - final AsyncCallbacks.ReadEntriesCallback callback; - InflightReadsLimiter.Handle newHandle = pendingReadsLimiter.acquire(estimatedReadSize, handle); - if (!newHandle.success) { - long now = System.currentTimeMillis(); - if (now - newHandle.creationTime > readEntryTimeoutMillis) { - String message = "Time-out elapsed while acquiring enough permits " - + "on the memory limiter to read from ledger " - + lh.getId() - + ", " + getName() - + ", estimated read size " + estimatedReadSize + " bytes" - + " for " + (1 + lastEntry - firstEntry) - + " entries (check managedLedgerMaxReadsInFlightSizeInMB)"; - log.error(message); - pendingReadsLimiter.release(newHandle); - originalCallback.readEntriesFailed( - new ManagedLedgerException.TooManyRequestsException(message), ctx); - return null; - } - ml.getExecutor().execute(() -> { - asyncReadEntry0WithLimits(lh, firstEntry, lastEntry, shouldCacheEntry, - originalCallback, ctx, newHandle, true); - }); - return null; - } else { - callback = new AsyncCallbacks.ReadEntriesCallback() { - - @Override - public void readEntriesComplete(List entries, Object ctx) { - if (!entries.isEmpty()) { - long size = entries.get(0).getLength(); - estimatedEntrySize = size; - - AtomicInteger remainingCount = new AtomicInteger(entries.size()); - for (Entry entry : entries) { - ((EntryImpl) entry).onDeallocate(() -> { - if (remainingCount.decrementAndGet() <= 0) { - pendingReadsLimiter.release(newHandle); - } - }); - } - } else { - pendingReadsLimiter.release(newHandle); - } - originalCallback.readEntriesComplete(entries, ctx); - } + return estimatedEntrySize + BOOKKEEPER_READ_OVERHEAD_PER_ENTRY; + } - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - pendingReadsLimiter.release(newHandle); - originalCallback.readEntriesFailed(exception, ctx); - } - }; - } - return callback; + private long getAvgEntrySize() { + long totalAddedEntriesCount = this.totalAddedEntriesCount.sum(); + long totalAddedEntriesSize = this.totalAddedEntriesSize.sum(); + return totalAddedEntriesCount != 0 ? totalAddedEntriesSize / totalAddedEntriesCount : 0; } /** @@ -442,8 +453,7 @@ CompletableFuture> readFromStorage(ReadHandle lh, try { // We got the entries, we need to transform them to a List<> type long totalSize = 0; - final List entriesToReturn = - Lists.newArrayListWithExpectedSize(entriesToRead); + final List entriesToReturn = new ArrayList<>(entriesToRead); for (LedgerEntry e : ledgerEntries) { EntryImpl entry = RangeEntryCacheManagerImpl.create(e, interceptor); entriesToReturn.add(entry); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java index 34be25df1f476..61d52aa3919ae 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java @@ -28,7 +28,9 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; +import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryMBeanImpl; @@ -57,12 +59,16 @@ public class RangeEntryCacheManagerImpl implements EntryCacheManager { private static final double evictionTriggerThresholdPercent = 0.98; - public RangeEntryCacheManagerImpl(ManagedLedgerFactoryImpl factory, OpenTelemetry openTelemetry) { - this.maxSize = factory.getConfig().getMaxCacheSize(); - this.inflightReadsLimiter = new InflightReadsLimiter( - factory.getConfig().getManagedLedgerMaxReadsInFlightSize(), openTelemetry); + public RangeEntryCacheManagerImpl(ManagedLedgerFactoryImpl factory, OrderedScheduler scheduledExecutor, + OpenTelemetry openTelemetry) { + ManagedLedgerFactoryConfig config = factory.getConfig(); + this.maxSize = config.getMaxCacheSize(); + this.inflightReadsLimiter = new InflightReadsLimiter(config.getManagedLedgerMaxReadsInFlightSize(), + config.getManagedLedgerMaxReadsInFlightPermitsAcquireQueueSize(), + config.getManagedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis(), + scheduledExecutor, openTelemetry); this.evictionTriggerThreshold = (long) (maxSize * evictionTriggerThresholdPercent); - this.cacheEvictionWatermark = factory.getConfig().getCacheEvictionWatermark(); + this.cacheEvictionWatermark = config.getCacheEvictionWatermark(); this.evictionPolicy = new EntryCacheDefaultEvictionPolicy(); this.mlFactory = factory; this.mlFactoryMBean = factory.getMbean(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java index b57dea6a5bb4d..48f0cf08ddff4 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java @@ -40,7 +40,6 @@ import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.awaitility.Awaitility; -import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; import org.mockito.stubbing.Answer; import org.testng.Assert; @@ -142,8 +141,8 @@ public void testPreciseLimitation(String missingCase) throws Exception { SimpleReadEntriesCallback cb0 = new SimpleReadEntriesCallback(); entryCache.asyncReadEntry(spyCurrentLedger, 125, 125, true, cb0, ctx); cb0.entries.join(); - Long sizePerEntry1 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); - Assert.assertEquals(sizePerEntry1, 1); + Long sizePerEntry1 = entryCache.getEstimatedEntrySize(); + Assert.assertEquals(sizePerEntry1, 1 + RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY); Awaitility.await().untilAsserted(() -> { long remainingBytes =limiter.getRemainingBytes(); Assert.assertEquals(remainingBytes, totalCapacity); @@ -179,8 +178,8 @@ public void testPreciseLimitation(String missingCase) throws Exception { Thread.sleep(3000); readCompleteSignal1.countDown(); cb1.entries.join(); - Long sizePerEntry2 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); - Assert.assertEquals(sizePerEntry2, 1); + Long sizePerEntry2 = entryCache.getEstimatedEntrySize(); + Assert.assertEquals(sizePerEntry2, 1 + RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY); long bytesAcquired2 = calculateBytesSizeBeforeFirstReading(readCount2, 1); long remainingBytesExpected2 = totalCapacity - bytesAcquired2; log.info("acquired : {}", bytesAcquired2); @@ -192,8 +191,8 @@ public void testPreciseLimitation(String missingCase) throws Exception { readCompleteSignal2.countDown(); cb2.entries.join(); - Long sizePerEntry3 = WhiteboxImpl.getInternalState(entryCache, "estimatedEntrySize"); - Assert.assertEquals(sizePerEntry3, 1); + Long sizePerEntry3 = entryCache.getEstimatedEntrySize(); + Assert.assertEquals(sizePerEntry3, 1 + RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY); Awaitility.await().untilAsserted(() -> { long remainingBytes = limiter.getRemainingBytes(); log.info("remainingBytes 2: {}", remainingBytes); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 04f8eecbe9a3b..5ec453a6d4e69 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -94,6 +94,7 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.PulsarMockBookKeeper; import org.apache.bookkeeper.client.PulsarMockLedgerHandle; +import org.apache.bookkeeper.client.PulsarMockReadHandleInterceptor; import org.apache.bookkeeper.client.api.LedgerEntries; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.ReadHandle; @@ -3133,17 +3134,26 @@ public void testManagedLedgerWithReadEntryTimeOut() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig().setReadEntryTimeoutSeconds(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("timeout_ledger_test", config); - BookKeeper bk = mock(BookKeeper.class); - doNothing().when(bk).asyncCreateLedger(anyInt(), anyInt(), anyInt(), any(), any(), any(), any(), any()); + Position position = ledger.addEntry("entry-1".getBytes()); + + // ensure that the read isn't cached + factory.getEntryCacheManager().clear(); + + bkc.setReadHandleInterceptor(new PulsarMockReadHandleInterceptor() { + @Override + public CompletableFuture interceptReadAsync(long ledgerId, long firstEntry, long lastEntry, + LedgerEntries entries) { + return CompletableFuture.supplyAsync(() -> { + return entries; + }, CompletableFuture.delayedExecutor(3, TimeUnit.SECONDS)); + } + }); + AtomicReference responseException1 = new AtomicReference<>(); String ctxStr = "timeoutCtx"; - CompletableFuture entriesFuture = new CompletableFuture<>(); - ReadHandle ledgerHandle = mock(ReadHandle.class); - doReturn(entriesFuture).when(ledgerHandle).readAsync(PositionFactory.EARLIEST.getLedgerId(), - PositionFactory.EARLIEST.getEntryId()); // (1) test read-timeout for: ManagedLedger.asyncReadEntry(..) - ledger.asyncReadEntry(ledgerHandle, PositionFactory.EARLIEST, new ReadEntryCallback() { + ledger.asyncReadEntry(position, new ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { responseException1.set(null); @@ -3155,18 +3165,20 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { responseException1.set(exception); } }, ctxStr); - ledger.asyncCreateLedger(bk, config, null, (rc, lh, ctx) -> {}, Collections.emptyMap()); - retryStrategically((test) -> responseException1.get() != null, 5, 1000); - assertNotNull(responseException1.get()); - assertTrue(responseException1.get().getMessage() - .startsWith(BKException.getMessage(BKException.Code.TimeoutException))); - // (2) test read-timeout for: ManagedLedger.asyncReadEntry(..) - AtomicReference responseException2 = new AtomicReference<>(); - Position readPositionRef = PositionFactory.EARLIEST; - ManagedCursorImpl cursor = new ManagedCursorImpl(bk, ledger, "cursor1"); - OpReadEntry opReadEntry = OpReadEntry.create(cursor, readPositionRef, 1, new ReadEntriesCallback() { + Awaitility.await().untilAsserted(() -> { + assertNotNull(responseException1.get()); + assertTrue(responseException1.get().getMessage() + .startsWith(BKException.getMessage(BKException.Code.TimeoutException))); + }); + // ensure that the read isn't cached + factory.getEntryCacheManager().clear(); + + // (2) test read-timeout for: ManagedCursor.asyncReadEntries(..) + AtomicReference responseException2 = new AtomicReference<>(); + ManagedCursor cursor = ledger.openCursor("cursor1", InitialPosition.Earliest); + cursor.asyncReadEntries(1, new ReadEntriesCallback() { @Override public void readEntriesComplete(List entries, Object ctx) { } @@ -3176,16 +3188,13 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { assertEquals(ctxStr, (String) ctx); responseException2.set(exception); } + }, ctxStr, PositionFactory.LATEST); - }, null, PositionFactory.LATEST, null); - ledger.asyncReadEntry(ledgerHandle, PositionFactory.EARLIEST.getEntryId(), PositionFactory.EARLIEST.getEntryId(), - opReadEntry, ctxStr); - retryStrategically((test) -> { - return responseException2.get() != null; - }, 5, 1000); - assertNotNull(responseException2.get()); - assertTrue(responseException2.get().getMessage() - .startsWith(BKException.getMessage(BKException.Code.TimeoutException))); + Awaitility.await().untilAsserted(() -> { + assertNotNull(responseException2.get()); + assertTrue(responseException2.get().getMessage() + .startsWith(BKException.getMessage(BKException.Code.TimeoutException))); + }); ledger.close(); } @@ -3723,6 +3732,10 @@ public void testInvalidateReadHandleWhenDeleteLedger() throws Exception { for (int i = 0; i < entries; i++) { ledger.addEntry(String.valueOf(i).getBytes(Encoding)); } + + // clear the cache to avoid flakiness + factory.getEntryCacheManager().clear(); + List entryList = cursor.readEntries(3); assertEquals(entryList.size(), 3); Awaitility.await().untilAsserted(() -> { @@ -3791,10 +3804,16 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception { for (int i = 0; i < entries; i++) { ledger.addEntry(String.valueOf(i).getBytes(Encoding)); } - List entryList = cursor.readEntries(3); - assertEquals(entryList.size(), 3); - assertEquals(ledger.ledgers.size(), 4); - assertEquals(ledger.ledgerCache.size(), 3); + + // clear the cache to avoid flakiness + factory.getEntryCacheManager().clear(); + + final List entryList = cursor.readEntries(3); + Awaitility.await().untilAsserted(() -> { + assertEquals(entryList.size(), 3); + assertEquals(ledger.ledgers.size(), 4); + assertEquals(ledger.ledgerCache.size(), 3); + }); cursor.clearBacklog(); cursor2.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); @@ -3803,11 +3822,17 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception { assertEquals(ledger.ledgerCache.size(), 0); }); + // clear the cache to avoid flakiness + factory.getEntryCacheManager().clear(); + // Verify the ReadHandle can be reopened. ManagedCursor cursor3 = ledger.openCursor("test-cursor3", InitialPosition.Earliest); - entryList = cursor3.readEntries(3); - assertEquals(entryList.size(), 3); - assertEquals(ledger.ledgerCache.size(), 3); + final List entryList2 = cursor3.readEntries(3); + Awaitility.await().untilAsserted(() -> { + assertEquals(entryList2.size(), 3); + assertEquals(ledger.ledgerCache.size(), 3); + }); + cursor3.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); Awaitility.await().untilAsserted(() -> { @@ -3815,7 +3840,6 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception { assertEquals(ledger.ledgerCache.size(), 0); }); - cursor.close(); cursor2.close(); cursor3.close(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java index 68135598e3339..7475b620f5792 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java @@ -21,40 +21,51 @@ import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; import static org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization.FREE; import static org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization.USED; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertTrue; +import static org.mockito.Mockito.mock; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.assertj.core.api.Assertions; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j public class InflightReadsLimiterTest { + private static final int ACQUIRE_QUEUE_SIZE = 1000; + private static final int ACQUIRE_TIMEOUT_MILLIS = 500; @DataProvider private static Object[][] isDisabled() { - return new Object[][] { - {0, true}, - {-1, true}, - {1, false}, + return new Object[][]{ + {0, true}, + {-1, true}, + {1, false}, }; } + @DataProvider + private static Object[] booleanValues() { + return new Object[]{ true, false }; + } + @Test(dataProvider = "isDisabled") public void testDisabled(long maxReadsInFlightSize, boolean shouldBeDisabled) throws Exception { var otel = buildOpenTelemetryAndReader(); @Cleanup var openTelemetry = otel.getLeft(); @Cleanup var metricReader = otel.getRight(); - var limiter = new InflightReadsLimiter(maxReadsInFlightSize, openTelemetry); - assertEquals(limiter.isDisabled(), shouldBeDisabled); + var limiter = new InflightReadsLimiter(maxReadsInFlightSize, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, + mock(ScheduledExecutorService.class), openTelemetry); + assertThat(limiter.isDisabled()).isEqualTo(shouldBeDisabled); if (shouldBeDisabled) { // Verify metrics are not present @@ -72,136 +83,459 @@ public void testBasicAcquireRelease() throws Exception { @Cleanup var openTelemetry = otel.getLeft(); @Cleanup var metricReader = otel.getRight(); - InflightReadsLimiter limiter = new InflightReadsLimiter(100, openTelemetry); - assertEquals(100, limiter.getRemainingBytes()); + InflightReadsLimiter limiter = + new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, + mock(ScheduledExecutorService.class), openTelemetry); + assertThat(limiter.getRemainingBytes()).isEqualTo(100); assertLimiterMetrics(metricReader, 100, 0, 100); - InflightReadsLimiter.Handle handle = limiter.acquire(100, null); - assertEquals(0, limiter.getRemainingBytes()); - assertTrue(handle.success); - assertEquals(handle.acquiredPermits, 100); - assertEquals(1, handle.trials); + Optional optionalHandle = limiter.acquire(100, null); + assertThat(limiter.getRemainingBytes()).isZero(); + assertThat(optionalHandle).isPresent(); + InflightReadsLimiter.Handle handle = optionalHandle.get(); + assertThat(handle.success()).isTrue(); + assertThat(handle.permits()).isEqualTo(100); assertLimiterMetrics(metricReader, 100, 100, 0); limiter.release(handle); - assertEquals(100, limiter.getRemainingBytes()); + assertThat(limiter.getRemainingBytes()).isEqualTo(100); assertLimiterMetrics(metricReader, 100, 0, 100); } - @Test public void testNotEnoughPermits() throws Exception { - InflightReadsLimiter limiter = new InflightReadsLimiter(100, OpenTelemetry.noop()); - assertEquals(100, limiter.getRemainingBytes()); - InflightReadsLimiter.Handle handle = limiter.acquire(100, null); - assertEquals(0, limiter.getRemainingBytes()); - assertTrue(handle.success); - assertEquals(handle.acquiredPermits, 100); - assertEquals(1, handle.trials); - - InflightReadsLimiter.Handle handle2 = limiter.acquire(100, null); - assertEquals(0, limiter.getRemainingBytes()); - assertFalse(handle2.success); - assertEquals(handle2.acquiredPermits, 0); - assertEquals(1, handle2.trials); + InflightReadsLimiter limiter = + new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, + mock(ScheduledExecutorService.class), OpenTelemetry.noop()); + assertThat(limiter.getRemainingBytes()).isEqualTo(100); + Optional optionalHandle = limiter.acquire(100, null); + assertThat(limiter.getRemainingBytes()).isZero(); + assertThat(optionalHandle).isPresent(); + InflightReadsLimiter.Handle handle = optionalHandle.get(); + assertThat(handle.success()).isTrue(); + assertThat(handle.permits()).isEqualTo(100); + + AtomicReference handle2Reference = new AtomicReference<>(); + Optional optionalHandle2 = limiter.acquire(100, handle2Reference::set); + assertThat(limiter.getRemainingBytes()).isZero(); + assertThat(optionalHandle2).isNotPresent(); limiter.release(handle); - assertEquals(100, limiter.getRemainingBytes()); - - handle2 = limiter.acquire(100, handle2); - assertEquals(0, limiter.getRemainingBytes()); - assertTrue(handle2.success); - assertEquals(handle2.acquiredPermits, 100); - assertEquals(2, handle2.trials); - - limiter.release(handle2); - assertEquals(100, limiter.getRemainingBytes()); + assertThat(handle2Reference) + .hasValueSatisfying(h -> + assertThat(h.success()).isTrue()); + limiter.release(handle2Reference.get()); + assertThat(limiter.getRemainingBytes()).isEqualTo(100); } @Test - public void testPartialAcquire() throws Exception { - InflightReadsLimiter limiter = new InflightReadsLimiter(100, OpenTelemetry.noop()); - assertEquals(100, limiter.getRemainingBytes()); - - InflightReadsLimiter.Handle handle = limiter.acquire(30, null); - assertEquals(70, limiter.getRemainingBytes()); - assertTrue(handle.success); - assertEquals(handle.acquiredPermits, 30); - assertEquals(1, handle.trials); - - InflightReadsLimiter.Handle handle2 = limiter.acquire(100, null); - assertEquals(0, limiter.getRemainingBytes()); - assertFalse(handle2.success); - assertEquals(handle2.acquiredPermits, 70); - assertEquals(1, handle2.trials); - - limiter.release(handle); + public void testAcquireTimeout() throws Exception { + @Cleanup("shutdownNow") + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + InflightReadsLimiter limiter = + new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, + executor, OpenTelemetry.noop()); + assertThat(limiter.getRemainingBytes()).isEqualTo(100); + limiter.acquire(100, null); + + AtomicReference handle2Reference = new AtomicReference<>(); + Optional optionalHandle2 = limiter.acquire(100, handle2Reference::set); + assertThat(optionalHandle2).isNotPresent(); + + Thread.sleep(ACQUIRE_TIMEOUT_MILLIS + 100); + + assertThat(handle2Reference).hasValueSatisfying(h -> assertThat(h.success()).isFalse()); + } - handle2 = limiter.acquire(100, handle2); - assertEquals(0, limiter.getRemainingBytes()); - assertTrue(handle2.success); - assertEquals(handle2.acquiredPermits, 100); - assertEquals(2, handle2.trials); + @Test + public void testMultipleQueuedEntriesWithExceptionInFirstCallback() throws Exception { + @Cleanup("shutdownNow") + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + InflightReadsLimiter limiter = + new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, + executor, OpenTelemetry.noop()); + assertThat(limiter.getRemainingBytes()) + .as("Initial remaining bytes should be 100") + .isEqualTo(100); + + // Acquire the initial permits + Optional handle1 = limiter.acquire(100, null); + assertThat(handle1) + .as("Initial handle should be present") + .isPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be 0 after acquiring 100 permits") + .isEqualTo(0); + + // Queue the first handle with a callback that throws an exception + AtomicReference handle2Reference = new AtomicReference<>(); + Optional handle2 = limiter.acquire(50, handle -> { + handle2Reference.set(handle); + throw new RuntimeException("Callback exception"); + }); + assertThat(handle2) + .as("Second handle should not be present") + .isNotPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0 after failed acquisition") + .isEqualTo(0); + + // Queue the second handle with a successful callback + AtomicReference handle3Reference = new AtomicReference<>(); + Optional handle3 = limiter.acquire(50, handle3Reference::set); + assertThat(handle3) + .as("Third handle should not be present as queue is full") + .isNotPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0") + .isEqualTo(0); + + // Release the initial handle to trigger the queued callbacks + limiter.release(handle1.get()); + + // Verify the first callback threw an exception but the second callback was handled successfully + assertThat(handle2Reference) + .as("Handle2 should have been set in the callback despite the exception") + .hasValueSatisfying(handle -> assertThat(handle.success()) + .as("Handle2 should be marked as successful") + .isTrue()); + assertThat(handle3Reference) + .as("Handle3 should have been set successfully") + .hasValueSatisfying(handle -> assertThat(handle.success()) + .as("Handle3 should be marked as successful") + .isTrue()); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0 after first releases are acquired") + .isEqualTo(0); + + // Release the second handle + limiter.release(handle3Reference.get()); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be 50 after releasing handle3") + .isEqualTo(50); + + // Release the third handle + limiter.release(handle3Reference.get()); + assertThat(limiter.getRemainingBytes()) + .as("All bytes should be released, so remaining bytes should be 100") + .isEqualTo(100); + } - limiter.release(handle2); - assertEquals(100, limiter.getRemainingBytes()); + @Test + public void testMultipleQueuedEntriesWithTimeoutAndExceptionInFirstCallback() throws Exception { + @Cleanup("shutdownNow") + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + InflightReadsLimiter limiter = + new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, + executor, OpenTelemetry.noop()); + assertThat(limiter.getRemainingBytes()) + .as("Initial remaining bytes should be 100") + .isEqualTo(100); + + // Acquire the initial permits + Optional handle1 = limiter.acquire(100, null); + assertThat(handle1) + .as("The first handle should be present after acquiring 100 permits") + .isPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be 0 after acquiring all permits") + .isEqualTo(0); + + // Queue the first handle with a callback that times out and throws an exception + AtomicReference handle2Reference = new AtomicReference<>(); + Optional handle2 = limiter.acquire(50, handle -> { + handle2Reference.set(handle); + throw new RuntimeException("Callback exception on timeout"); + }); + assertThat(handle2) + .as("The second handle should not be present as the callback throws an exception") + .isNotPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0 after failed acquisition") + .isEqualTo(0); + + // Introduce a delay to differentiate operations between queued entries + Thread.sleep(50); + + // Queue the second handle with a successful callback + AtomicReference handle3Reference = new AtomicReference<>(); + Optional handle3 = limiter.acquire(50, handle3Reference::set); + assertThat(handle3) + .as("The third handle should not be present as permits are still unavailable") + .isNotPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0 after failed acquisition attempt") + .isEqualTo(0); + + // Wait for the timeout to occur + Thread.sleep(ACQUIRE_TIMEOUT_MILLIS + 100); + + // Verify the first callback timed out and threw an exception, and the second callback was handled + assertThat(handle2Reference) + .as("Handle2 should have been set in the callback despite the exception") + .hasValueSatisfying(handle -> assertThat(handle.success()) + .as("Handle2 should be marked as unsuccessful due to a timeout") + .isFalse()); + assertThat(handle3Reference) + .as("Handle3 should have been set in the callback after the permits became available") + .hasValueSatisfying(handle -> Assertions.assertThat(handle.success()) + .as("Handle3 should be marked as unsuccessful due to a timeout") + .isFalse()); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0 as no permits were released") + .isEqualTo(0); + + // Release the first handle + limiter.release(handle1.get()); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be fully restored to 100 after releasing all permits") + .isEqualTo(100); + } + @Test + public void testMultipleQueuedEntriesWithTimeoutsThatAreTimedOutWhenPermitsAreAvailable() throws Exception { + // Use a mock executor to simulate scenarios where timed out queued handles are processed when permits become + // available + ScheduledExecutorService executor = mock(ScheduledExecutorService.class); + InflightReadsLimiter limiter = + new InflightReadsLimiter(100, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, + executor, OpenTelemetry.noop()); + assertThat(limiter.getRemainingBytes()) + .as("Initial remaining bytes should be 100") + .isEqualTo(100); + + // Acquire the initial permits + Optional handle1 = limiter.acquire(100, null); + assertThat(handle1) + .as("The first handle should be present after acquiring 100 permits") + .isPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be 0 after acquiring all permits") + .isEqualTo(0); + + // Queue the first handle + AtomicReference handle2Reference = new AtomicReference<>(); + Optional handle2 = limiter.acquire(50, handle2Reference::set); + assertThat(handle2) + .as("The second handle should not be present as permits are unavailable") + .isNotPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0 after failed acquisition attempt for handle2") + .isEqualTo(0); + + // Queue the second handle + AtomicReference handle3Reference = new AtomicReference<>(); + Optional handle3 = limiter.acquire(50, handle3Reference::set); + assertThat(handle3) + .as("The third handle should not be present as permits are unavailable") + .isNotPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0 after failed acquisition attempt for handle3") + .isEqualTo(0); + + // Wait for the timeout to occur + Thread.sleep(ACQUIRE_TIMEOUT_MILLIS + 100); + + // Queue another handle + AtomicReference handle4Reference = new AtomicReference<>(); + Optional handle4 = limiter.acquire(50, handle4Reference::set); + assertThat(handle4) + .as("The fourth handle should not be present because permits are unavailable") + .isNotPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0 after failed acquisition attempt for handle4") + .isEqualTo(0); + + // Queue another handle + AtomicReference handle5Reference = new AtomicReference<>(); + Optional handle5 = limiter.acquire(100, handle5Reference::set); + assertThat(handle5) + .as("The fifth handle should not be present as permits are unavailable") + .isNotPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should still be 0 after failed acquisition attempt for handle5") + .isEqualTo(0); + + // Release the first handle + limiter.release(handle1.get()); + + assertThat(handle2Reference) + .as("Handle2 should have been set in the callback and marked unsuccessful") + .hasValueSatisfying(handle -> assertThat(handle.success()).isFalse()); + + assertThat(handle3Reference) + .as("Handle3 should have been set in the callback and marked unsuccessful") + .hasValueSatisfying(handle -> assertThat(handle.success()).isFalse()); + + assertThat(handle4Reference) + .as("Handle4 should have been set in the callback and marked successful") + .hasValueSatisfying(handle -> assertThat(handle.success()).isTrue()); + + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be 50 after releasing handle4") + .isEqualTo(50); + + limiter.release(handle4Reference.get()); + + assertThat(handle5Reference) + .as("Handle5 should have been set in the callback and marked successful") + .hasValueSatisfying(handle -> assertThat(handle.success()).isTrue()); + + limiter.release(handle5Reference.get()); + + assertThat(limiter.getRemainingBytes()) + .as("All bytes should be released, so remaining bytes should be back to 100") + .isEqualTo(100); } @Test - public void testTooManyTrials() throws Exception { - InflightReadsLimiter limiter = new InflightReadsLimiter(100, OpenTelemetry.noop()); - assertEquals(100, limiter.getRemainingBytes()); - - InflightReadsLimiter.Handle handle = limiter.acquire(30, null); - assertEquals(70, limiter.getRemainingBytes()); - assertTrue(handle.success); - assertEquals(handle.acquiredPermits, 30); - assertEquals(1, handle.trials); - - InflightReadsLimiter.Handle handle2 = limiter.acquire(100, null); - assertEquals(0, limiter.getRemainingBytes()); - assertFalse(handle2.success); - assertEquals(handle2.acquiredPermits, 70); - assertEquals(1, handle2.trials); - - handle2 = limiter.acquire(100, handle2); - assertEquals(0, limiter.getRemainingBytes()); - assertFalse(handle2.success); - assertEquals(handle2.acquiredPermits, 70); - assertEquals(2, handle2.trials); - - handle2 = limiter.acquire(100, handle2); - assertEquals(0, limiter.getRemainingBytes()); - assertFalse(handle2.success); - assertEquals(handle2.acquiredPermits, 70); - assertEquals(3, handle2.trials); - - handle2 = limiter.acquire(100, handle2); - assertEquals(0, limiter.getRemainingBytes()); - assertFalse(handle2.success); - assertEquals(handle2.acquiredPermits, 70); - assertEquals(4, handle2.trials); - - // too many trials, start from scratch - handle2 = limiter.acquire(100, handle2); - assertEquals(70, limiter.getRemainingBytes()); - assertFalse(handle2.success); - assertEquals(handle2.acquiredPermits, 0); - assertEquals(1, handle2.trials); + public void testQueueSizeLimitReached() throws Exception { + @Cleanup("shutdownNow") + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + + // Minimum queue size is 4. + final int queueSizeLimit = 4; + InflightReadsLimiter limiter = + new InflightReadsLimiter(100, queueSizeLimit, ACQUIRE_TIMEOUT_MILLIS, executor, OpenTelemetry.noop()); + + assertThat(limiter.getRemainingBytes()) + .as("Initial remaining bytes should be 100") + .isEqualTo(100); + + // Acquire all available permits (consume 100 bytes) + Optional handle1 = limiter.acquire(100, null); + assertThat(handle1) + .as("The first handle should be present after acquiring all available permits") + .isPresent() + .hasValueSatisfying(handle -> assertThat(handle.success()).isTrue()); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be zero after acquiring all permits") + .isEqualTo(0); + + // Queue up to the limit (4 requests) + AtomicReference handle2Reference = new AtomicReference<>(); + assertThat(limiter.acquire(50, handle2Reference::set)).isNotPresent(); + + AtomicReference handle3Reference = new AtomicReference<>(); + assertThat(limiter.acquire(50, handle3Reference::set)).isNotPresent(); + + AtomicReference handle4Reference = new AtomicReference<>(); + assertThat(limiter.acquire(50, handle4Reference::set)).isNotPresent(); + + AtomicReference handle5Reference = new AtomicReference<>(); + assertThat(limiter.acquire(50, handle5Reference::set)).isNotPresent(); + + // Attempt to add one more request, which should fail as the queue is full + Optional handle6 = limiter.acquire(50, null); + assertThat(handle6) + .as("The sixth handle should not be successfull since the queue is full") + .hasValueSatisfying(handle -> assertThat(handle.success()).isFalse()); + } - limiter.release(handle); + @Test(dataProvider = "booleanValues") + public void testAcquireExceedingMaxReadsInFlightSize(boolean firstInQueue) throws Exception { + @Cleanup("shutdownNow") + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + + long maxReadsInFlightSize = 100; + InflightReadsLimiter limiter = + new InflightReadsLimiter(maxReadsInFlightSize, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, executor, + OpenTelemetry.noop()); + + // Initial state + assertThat(limiter.getRemainingBytes()) + .as("Initial remaining bytes should match maxReadsInFlightSize") + .isEqualTo(maxReadsInFlightSize); + + // Acquire all permits (consume 100 bytes) + Optional handle1 = limiter.acquire(100, null); + assertThat(handle1) + .as("The first handle should be present") + .isPresent(); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be zero after acquiring all permits") + .isEqualTo(0); + + + AtomicReference handle2Reference = new AtomicReference<>(); + + if (!firstInQueue) { + Optional handle2 = limiter.acquire(50, handle2Reference::set); + assertThat(handle2) + .as("The second handle should not be present as remaining permits are zero") + .isNotPresent(); + } - handle2 = limiter.acquire(100, handle2); - assertEquals(0, limiter.getRemainingBytes()); - assertTrue(handle2.success); - assertEquals(handle2.acquiredPermits, 100); - assertEquals(2, handle2.trials); + // Attempt to acquire more than maxReadsInFlightSize while all permits are in use + AtomicReference handleExceedingMaxReference = new AtomicReference<>(); + Optional handleExceedingMaxOptional = + limiter.acquire(200, handleExceedingMaxReference::set); + assertThat(handleExceedingMaxOptional) + .as("The second handle should not be present as remaining permits are zero") + .isNotPresent(); + + // Release handle1 permits + limiter.release(handle1.get()); + + if (!firstInQueue) { + assertThat(handle2Reference) + .as("Handle2 should have been set in the callback and marked successful") + .hasValueSatisfying(handle -> { + assertThat(handle.success()).isTrue(); + assertThat(handle.permits()).isEqualTo(50); + }); + limiter.release(handle2Reference.get()); + } + + assertThat(handleExceedingMaxReference) + .as("Handle2 should have been set in the callback and marked successful") + .hasValueSatisfying(handle -> { + assertThat(handle.success()).isTrue(); + assertThat(handle.permits()).isEqualTo(maxReadsInFlightSize); + }); - limiter.release(handle2); - assertEquals(100, limiter.getRemainingBytes()); + limiter.release(handleExceedingMaxReference.get()); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be fully replenished after releasing all permits") + .isEqualTo(maxReadsInFlightSize); + } + + @Test + public void testAcquireExceedingMaxReadsWhenAllPermitsAvailable() throws Exception { + @Cleanup("shutdownNow") + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + + long maxReadsInFlightSize = 100; + InflightReadsLimiter limiter = + new InflightReadsLimiter(maxReadsInFlightSize, ACQUIRE_QUEUE_SIZE, ACQUIRE_TIMEOUT_MILLIS, executor, + OpenTelemetry.noop()); + + // Initial state + assertThat(limiter.getRemainingBytes()) + .as("Initial remaining bytes should match maxReadsInFlightSize") + .isEqualTo(maxReadsInFlightSize); + + // Acquire permits > maxReadsInFlightSize + Optional handleExceedingMaxOptional = + limiter.acquire(2 * maxReadsInFlightSize, null); + assertThat(handleExceedingMaxOptional) + .as("The handle for exceeding max permits should be present") + .hasValueSatisfying(handle -> { + assertThat(handle.success()).isTrue(); + assertThat(handle.permits()).isEqualTo(maxReadsInFlightSize); + }); + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be zero after acquiring all permits") + .isEqualTo(0); + + // Release permits + limiter.release(handleExceedingMaxOptional.get()); + + assertThat(limiter.getRemainingBytes()) + .as("Remaining bytes should be fully replenished after releasing all permits") + .isEqualTo(maxReadsInFlightSize); } private Pair buildOpenTelemetryAndReader() { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java index 383568c17e83d..55068580f62f1 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java @@ -35,6 +35,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import lombok.Data; import lombok.extern.slf4j.Slf4j; @@ -91,7 +92,8 @@ void setupMocks() { config.setReadEntryTimeoutSeconds(10000); when(rangeEntryCache.getName()).thenReturn("my-topic"); when(rangeEntryCache.getManagedLedgerConfig()).thenReturn(config); - inflighReadsLimiter = new InflightReadsLimiter(0, OpenTelemetry.noop()); + inflighReadsLimiter = new InflightReadsLimiter(0, 0, 0, + mock(ScheduledExecutorService.class), OpenTelemetry.noop()); when(rangeEntryCache.getPendingReadsLimiter()).thenReturn(inflighReadsLimiter); pendingReadsManager = new PendingReadsManager(rangeEntryCache); doAnswer(new Answer() { diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index ce3b25571e271..37ef1cfaf62ae 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2114,6 +2114,15 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + " Consumer Netty channel. Use O to disable") private long managedLedgerMaxReadsInFlightSizeInMB = 0; + @FieldContext(category = CATEGORY_STORAGE_ML, doc = "Maximum time to wait for acquiring permits for max reads in " + + "flight when managedLedgerMaxReadsInFlightSizeInMB is set (>0) and the limit is reached.") + private long managedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis = 60000; + + @FieldContext(category = CATEGORY_STORAGE_ML, doc = "Maximum number of reads that can be queued for acquiring " + + "permits for max reads in flight when managedLedgerMaxReadsInFlightSizeInMB is set (>0) and the limit " + + "is reached.") + private int managedLedgerMaxReadsInFlightPermitsAcquireQueueSize = 50000; + @FieldContext( category = CATEGORY_STORAGE_ML, dynamic = true, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java index 3d945afe4c115..b060475a43f31 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java @@ -72,8 +72,21 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata managedLedgerFactoryConfig.setCacheEvictionTimeThresholdMillis( conf.getManagedLedgerCacheEvictionTimeThresholdMillis()); managedLedgerFactoryConfig.setCopyEntriesInCache(conf.isManagedLedgerCacheCopyEntries()); - managedLedgerFactoryConfig.setManagedLedgerMaxReadsInFlightSize( - conf.getManagedLedgerMaxReadsInFlightSizeInMB() * 1024L * 1024L); + long managedLedgerMaxReadsInFlightSizeBytes = conf.getManagedLedgerMaxReadsInFlightSizeInMB() * 1024L * 1024L; + if (managedLedgerMaxReadsInFlightSizeBytes > 0 && conf.getDispatcherMaxReadSizeBytes() > 0 + && managedLedgerMaxReadsInFlightSizeBytes < conf.getDispatcherMaxReadSizeBytes()) { + log.warn("Invalid configuration for managedLedgerMaxReadsInFlightSizeInMB: {}, " + + "dispatcherMaxReadSizeBytes: {}. managedLedgerMaxReadsInFlightSizeInMB in bytes should " + + "be greater than dispatcherMaxReadSizeBytes. You should set " + + "managedLedgerMaxReadsInFlightSizeInMB to at least {}", + conf.getManagedLedgerMaxReadsInFlightSizeInMB(), conf.getDispatcherMaxReadSizeBytes(), + (conf.getDispatcherMaxReadSizeBytes() / (1024L * 1024L)) + 1); + } + managedLedgerFactoryConfig.setManagedLedgerMaxReadsInFlightSize(managedLedgerMaxReadsInFlightSizeBytes); + managedLedgerFactoryConfig.setManagedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis( + conf.getManagedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis()); + managedLedgerFactoryConfig.setManagedLedgerMaxReadsInFlightPermitsAcquireQueueSize( + conf.getManagedLedgerMaxReadsInFlightPermitsAcquireQueueSize()); managedLedgerFactoryConfig.setPrometheusStatsLatencyRolloverSeconds( conf.getManagedLedgerPrometheusStatsLatencyRolloverSeconds()); managedLedgerFactoryConfig.setTraceTaskExecution(conf.isManagedLedgerTraceTaskExecution()); From 12b057999f5674ae6da6b04ff8bd212d1904cc88 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 29 Jan 2025 18:28:39 -0800 Subject: [PATCH 283/327] [fix] Avoid NPE when closing an uninitialized SameAuthParamsLookupAutoClusterFailover (#23911) --- .../pulsar/client/impl/PulsarClientImpl.java | 3 +++ .../SameAuthParamsLookupAutoClusterFailover.java | 14 ++++++++++++-- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 871666620b7b4..9d1e45428e9df 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -262,6 +262,9 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG this::reduceConsumerReceiverQueueSize); state.set(State.Open); } catch (Throwable t) { + // Log the exception first, or it could be missed if there are any subsequent exceptions in the + // shutdown sequence + log.error("Failed to create Pulsar client instance.", t); shutdown(); shutdownEventLoopGroup(eventLoopGroupReference); closeCnxPool(connectionPoolReference); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java index 4beff4719c895..d71ac095a3b62 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java @@ -105,11 +105,21 @@ public String getServiceUrl() { @Override public void close() throws Exception { + if (closed) { + return; + } + log.info("Closing service url provider. Current pulsar service: [{}] {}", currentPulsarServiceIndex, pulsarServiceUrlArray[currentPulsarServiceIndex]); + if (scheduledCheckTask != null) { + scheduledCheckTask.cancel(false); + } + + if (executor != null) { + executor.shutdownNow(); + } + closed = true; - scheduledCheckTask.cancel(false); - executor.shutdownNow(); } private int firstHealthyPulsarService() { From cdab2d6dc4b620bdf129b05882b5f3bf4925ff61 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 30 Jan 2025 18:23:37 -0800 Subject: [PATCH 284/327] [fix] Initialize UrlServiceProvider before trying to use transaction coordinator (#23914) --- .../org/apache/pulsar/client/impl/ClientBuilderImpl.java | 6 +----- .../org/apache/pulsar/client/impl/PulsarClientImpl.java | 4 ++++ 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java index 7652d8a41994d..02140fca8ab8f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java @@ -68,11 +68,7 @@ public PulsarClient build() throws PulsarClientException { if (conf.getAuthentication() == null || conf.getAuthentication() == AuthenticationDisabled.INSTANCE) { setAuthenticationFromPropsIfAvailable(conf); } - PulsarClient client = new PulsarClientImpl(conf); - if (conf.getServiceUrlProvider() != null) { - conf.getServiceUrlProvider().initialize(client); - } - return client; + return new PulsarClientImpl(conf); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 9d1e45428e9df..38df40dee400e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -247,6 +247,10 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG this.timer = timer; } + if (conf.getServiceUrlProvider() != null) { + conf.getServiceUrlProvider().initialize(this); + } + if (conf.isEnableTransaction()) { tcClient = new TransactionCoordinatorClientImpl(this); try { From 144fe2eb634471d338a7e6ee879bc6175e5d061c Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Thu, 6 Feb 2025 10:07:16 +0800 Subject: [PATCH 285/327] [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 --- .../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 2415930a99a6c..fd26750bafab4 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 @@ -1729,7 +1729,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 8a40b30cf47a91ec02d931e6371d02409ba5951e Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 6 Feb 2025 10:14:28 +0800 Subject: [PATCH 286/327] [fix][broker] Closed topics won't be removed from the cache (#23884) --- .../pulsar/broker/service/AbstractTopic.java | 8 + .../pulsar/broker/service/BrokerService.java | 51 ++---- .../buffer/impl/TopicTransactionBuffer.java | 27 +++- .../impl/TransactionPersistentTopicTest.java | 148 ++++++++++++++++++ 4 files changed, 188 insertions(+), 46 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/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index 69a38bc50de9d..9a115e6d1ca4f 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 @@ -48,6 +48,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; @@ -96,6 +97,13 @@ public abstract class AbstractTopic implements Topic, TopicPolicyListener { protected final String topic; + // Reference to the CompletableFuture returned when creating this topic in BrokerService. + // Used to safely remove the topic from BrokerService's cache by ensuring we remove the exact + // topic instance that was created. + @Getter + @Setter + protected volatile CompletableFuture> 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 79e6fb2b02e31..ddd436b085493 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 @@ -1326,6 +1326,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); @@ -1800,6 +1801,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()) @@ -2409,47 +2411,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 41977e6b61d88..c43f0ed7fb9c1 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 @@ -109,7 +109,25 @@ public class TopicTransactionBuffer extends TopicTransactionBufferState implemen private final AbortedTxnProcessor.SnapshotType snapshotType; private final MaxReadPositionCallBack maxReadPositionCallBack; + private static AbortedTxnProcessor createSnapshotProcessor(PersistentTopic topic) { + return topic.getBrokerService().getPulsar().getConfiguration().isTransactionBufferSegmentedSnapshotEnabled() + ? new SnapshotSegmentAbortedTxnProcessorImpl(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), determineSnapshotType(topic)); + } + + @VisibleForTesting + TopicTransactionBuffer(PersistentTopic topic, AbortedTxnProcessor snapshotAbortedTxnProcessor, + AbortedTxnProcessor.SnapshotType snapshotType) { super(State.None); this.topic = topic; this.timer = topic.getBrokerService().getPulsar().getTransactionTimer(); @@ -118,13 +136,8 @@ public TopicTransactionBuffer(PersistentTopic topic) { this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar() .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis(); this.maxReadPosition = 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.snapshotType = snapshotType; 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..508423adce4d8 --- /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, AbortedTxnProcessor.SnapshotType.Single); + }; + 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); + } + } + +} From 99dc74b0e268e549384ef2099f17c773101a79bb Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Thu, 6 Feb 2025 17:56:57 +0800 Subject: [PATCH 287/327] [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. --- .../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 4d890a3d5db4d..dc92aeb0c7703 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 @@ -624,7 +624,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; }); @@ -885,6 +890,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 @@ -892,6 +901,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 36150ee21b32c..08c99085f80e7 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 @@ -47,6 +47,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; @@ -154,7 +155,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 a8e5e7a3ce77b..538731eda4315 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 @@ -346,7 +346,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); @@ -946,7 +946,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 a19eaa2b972ab8fe1a2a45e65df1566cf28fb336 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 6 Feb 2025 18:52:08 +0800 Subject: [PATCH 288/327] [improve][client] Avoid logging errors for retriable errors when creating producer (#23935) --- .../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 304b84961b7fa..fb2246f3a66a1 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 @@ -1917,14 +1917,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) -> { @@ -1936,6 +1928,7 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { future.complete(null); return null; } + if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededException) { synchronized (this) { log.warn("[{}] [{}] Topic backlog quota exceeded. Throwing Exception on producer.", topic, @@ -1954,6 +1947,10 @@ 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 5e5d514174fdbc1b400df51fafaa18110f1c31a9 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Thu, 6 Feb 2025 21:03:10 +0800 Subject: [PATCH 289/327] [improve][broker] Avoid logging errors when there is a connection issue during subscription. (#23939) --- .../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 d30dfc319e098..43a95cc86016f 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 @@ -218,6 +218,12 @@ public ConsumerAssignException(String msg) { } } + 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 7cdc8cc11a482..1931a09497e3f 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 @@ -351,7 +351,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 e920c483bb3ea..ed05e47ed38e3 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 @@ -1031,7 +1031,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()) { @@ -1068,6 +1069,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 3c0bbee91368086189816c26357491e4fe596e01 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Fri, 7 Feb 2025 09:48:47 +0800 Subject: [PATCH 290/327] [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 --- .../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 fd26750bafab4..d33264b171a18 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 @@ -1627,8 +1627,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 e6cfd2bbcc536152bb95ce2bffdff26e34bce73f Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Sat, 8 Feb 2025 13:18:46 +0800 Subject: [PATCH 291/327] [improve][io] Allow skipping connector deployment (#23932) Signed-off-by: Zixuan Liu --- 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 da3d8479b1fe4..b24577cfb830c 100644 --- a/pom.xml +++ b/pom.xml @@ -133,6 +133,7 @@ flexible messaging model and an intuitive client API. latest false false + false package package diff --git a/pulsar-io/common/pom.xml b/pulsar-io/common/pom.xml index 987c6c8cc8b75..7eef536b2b6b6 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 8f7920c7255ed..fa4236bf870fe 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 4eba7553883f1..968eb81b24614 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 54e9eb19d50b08978c953ec060226fd6c2a54775 Mon Sep 17 00:00:00 2001 From: Yuri Mizushima Date: Sat, 8 Feb 2025 19:03:29 +0900 Subject: [PATCH 292/327] [feat][client] Support forward proxy for the ZTS server in pulsar-client-auth-athenz (#23947) --- 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 b24577cfb830c..ac1d68647c56d 100644 --- a/pom.xml +++ b/pom.xml @@ -152,7 +152,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 f8e4c11b5cd94382a3493b3e129e46bfc6a0621d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Feb 2025 19:01:07 +0200 Subject: [PATCH 293/327] [fix][broker] Fix rate limiter token bucket and clock consistency issues causing excessive throttling and connection timeouts (#23930) --- microbench/README.md | 26 ++ .../broker/qos/AsyncTokenBucketBenchmark.java | 17 +- ...efaultMonotonicSnapshotClockBenchmark.java | 102 +++++++ .../pulsar/broker/qos/AsyncTokenBucket.java | 127 +++++---- .../broker/qos/AsyncTokenBucketBuilder.java | 34 +++ .../qos/DefaultMonotonicSnapshotClock.java | 260 +++++++++++++++--- .../qos/DynamicRateAsyncTokenBucket.java | 7 +- .../DynamicRateAsyncTokenBucketBuilder.java | 6 +- .../broker/qos/FinalRateAsyncTokenBucket.java | 7 +- .../qos/FinalRateAsyncTokenBucketBuilder.java | 2 +- .../pulsar/broker/service/BrokerService.java | 6 + .../service/PublishRateLimiterImpl.java | 11 +- .../persistent/DispatchRateLimiter.java | 39 ++- .../persistent/SubscribeRateLimiter.java | 8 +- .../auth/MockedPulsarServiceBaseTest.java | 46 ++-- .../broker/qos/AsyncTokenBucketTest.java | 143 +++++++++- .../DefaultMonotonicSnapshotClockTest.java | 185 +++++++++++++ .../RGUsageMTAggrWaitForAllMsgsTest.java | 5 +- .../service/PublishRateLimiterTest.java | 5 +- ...AbstractMessageDispatchThrottlingTest.java | 116 ++++++++ .../api/MessageDispatchThrottlingTest.java | 166 +++-------- ...criptionMessageDispatchThrottlingTest.java | 57 ++-- .../impl/MessagePublishThrottlingTest.java | 2 +- 23 files changed, 1090 insertions(+), 287 deletions(-) create mode 100644 microbench/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockBenchmark.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java diff --git a/microbench/README.md b/microbench/README.md index 780e3a5a1d3e8..f50c3036ff494 100644 --- a/microbench/README.md +++ b/microbench/README.md @@ -41,3 +41,29 @@ For fast recompiling of the benchmarks (without compiling Pulsar modules) and cr mvn -Pmicrobench -pl microbench clean package ``` +### Running specific benchmarks + +Display help: + +```shell +java -jar microbench/target/microbenchmarks.jar -h +``` + +Listing all benchmarks: + +```shell +java -jar microbench/target/microbenchmarks.jar -l +``` + +Running specific benchmarks: + +```shell +java -jar microbench/target/microbenchmarks.jar ".*BenchmarkName.*" +``` + +Checking what benchmarks match the pattern: + +```shell +java -jar microbench/target/microbenchmarks.jar ".*BenchmarkName.*" -lp +``` + diff --git a/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java b/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java index 4c069e72ea3ba..1b210258f13d2 100644 --- a/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java +++ b/microbench/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBenchmark.java @@ -33,6 +33,7 @@ import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Threads; import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; @Fork(3) @BenchmarkMode(Mode.Throughput) @@ -59,23 +60,29 @@ public void teardown() { @Benchmark @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) - public void consumeTokensBenchmark001Threads() { - asyncTokenBucket.consumeTokens(1); + public void consumeTokensBenchmark001Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole); } @Threads(10) @Benchmark @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) - public void consumeTokensBenchmark010Threads() { - asyncTokenBucket.consumeTokens(1); + public void consumeTokensBenchmark010Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole); } @Threads(100) @Benchmark @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) - public void consumeTokensBenchmark100Threads() { + public void consumeTokensBenchmark100Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole); + } + + private void consumeTokenAndGetTokens(Blackhole blackhole) { asyncTokenBucket.consumeTokens(1); + // blackhole is used to ensure that the compiler doesn't do dead code elimination + blackhole.consume(asyncTokenBucket.getTokens()); } } diff --git a/microbench/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockBenchmark.java b/microbench/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockBenchmark.java new file mode 100644 index 0000000000000..d9054b8fe4be8 --- /dev/null +++ b/microbench/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockBenchmark.java @@ -0,0 +1,102 @@ +/* + * 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.qos; + +import java.util.concurrent.TimeUnit; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(3) +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.SECONDS) +@State(Scope.Thread) +public class DefaultMonotonicSnapshotClockBenchmark { + private DefaultMonotonicSnapshotClock monotonicSnapshotClock = + new DefaultMonotonicSnapshotClock(TimeUnit.MILLISECONDS.toNanos(1), System::nanoTime); + + @TearDown(Level.Iteration) + public void teardown() { + monotonicSnapshotClock.close(); + } + + @Threads(1) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanos001Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, false); + } + + @Threads(10) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanos010Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, false); + } + + @Threads(100) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanos100Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, false); + } + + @Threads(1) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanosRequestSnapshot001Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, true); + } + + @Threads(10) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanosRequestSnapshot010Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, true); + } + + @Threads(100) + @Benchmark + @Measurement(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + @Warmup(time = 10, timeUnit = TimeUnit.SECONDS, iterations = 1) + public void getTickNanosRequestSnapshot100Threads(Blackhole blackhole) { + consumeTokenAndGetTokens(blackhole, true); + } + + private void consumeTokenAndGetTokens(Blackhole blackhole, boolean requestSnapshot) { + // blackhole is used to ensure that the compiler doesn't do dead code elimination + blackhole.consume(monotonicSnapshotClock.getTickNanos(requestSnapshot)); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java index ac9a1f03e592b..8c43fa0a816fa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucket.java @@ -42,6 +42,10 @@ * connection or client from the throttling queue to unthrottle. Before unthrottling, the application should check * for available tokens. If tokens are still not available, the application should continue with throttling and * repeat the throttling loop. + *

By default, the AsyncTokenBucket is eventually consistent. This means that the token balance is updated + * with added tokens and consumed tokens at most once during each "increment", when time advances more than the + * configured resolution. There are settings for configuring consistency, please see {@link AsyncTokenBucketBuilder} + * for details. *

This class does not produce side effects outside its own scope. It functions similarly to a stateful function, * akin to a counter function. In essence, it is a sophisticated counter. It can serve as a foundational component for * constructing higher-level asynchronous rate limiter implementations, which require side effects for throttling. @@ -119,9 +123,28 @@ public static void resetToDefaultEventualConsistentTokensView() { */ private final LongAdder pendingConsumedTokens = new LongAdder(); - protected AsyncTokenBucket(MonotonicSnapshotClock clockSource, long resolutionNanos) { + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the consumed tokens are subtracted from + * the total amount of tokens at most once during each "increment", when time advances more than the configured + * resolution. This setting determines if the consumed tokens are subtracted from tokens balance consistently. + * For high performance, it is recommended to keep this setting as false. + */ + private final boolean consistentConsumedTokens; + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the added tokens are calculated based + * on elapsed time at most once during each "increment", when time advances more than the configured + * resolution. This setting determines if the added tokens are calculated and added to tokens balance consistently. + * For high performance, it is recommended to keep this setting as false. + */ + private final boolean consistentAddedTokens; + + protected AsyncTokenBucket(MonotonicSnapshotClock clockSource, long resolutionNanos, + boolean consistentConsumedTokens, boolean consistentAddedTokens) { this.clockSource = clockSource; this.resolutionNanos = resolutionNanos; + this.lastNanos = Long.MIN_VALUE; + this.consistentConsumedTokens = consistentConsumedTokens; + this.consistentAddedTokens = consistentAddedTokens; } public static FinalRateAsyncTokenBucketBuilder builder() { @@ -139,36 +162,46 @@ public static DynamicRateAsyncTokenBucketBuilder builderForDynamicRate() { /** * Consumes tokens and possibly updates the tokens balance. New tokens are calculated and added to the current * tokens balance each time the update takes place. The update takes place once in every interval of the configured - * resolutionNanos or when the forceUpdateTokens parameter is true. + * resolutionNanos or when the forceConsistentTokens parameter is true. * When the tokens balance isn't updated, the consumed tokens are added to the pendingConsumedTokens LongAdder * counter which gets flushed the next time the tokens are updated. This makes the tokens balance * eventually consistent. The reason for this design choice is to optimize performance by preventing CAS loop * contention which could cause excessive CPU consumption. * * @param consumeTokens number of tokens to consume, can be 0 to update the tokens balance - * @param forceUpdateTokens if true, the tokens are updated even if the configured resolution hasn't passed + * @param forceConsistentTokens if true, the token balance is updated consistently * @return the current number of tokens in the bucket or Long.MIN_VALUE when the number of tokens is unknown due * to eventual consistency */ - private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolean forceUpdateTokens) { + private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolean forceConsistentTokens) { if (consumeTokens < 0) { throw new IllegalArgumentException("consumeTokens must be >= 0"); } - long currentNanos = clockSource.getTickNanos(forceUpdateTokens); + boolean requestConsistentTickNanosSnapshot = + consistentAddedTokens || consistentConsumedTokens || forceConsistentTokens || resolutionNanos == 0; + long currentNanos = clockSource.getTickNanos(requestConsistentTickNanosSnapshot); + long newTokens = 0; // check if the tokens should be updated immediately - if (shouldUpdateTokensImmediately(currentNanos, forceUpdateTokens)) { + if (shouldAddTokensImmediately(currentNanos, forceConsistentTokens)) { // calculate the number of new tokens since the last update - long newTokens = calculateNewTokensSinceLastUpdate(currentNanos); - // calculate the total amount of tokens to consume in this update + newTokens = calculateNewTokensSinceLastUpdate(currentNanos, forceConsistentTokens); + } + // update tokens if there are new tokens or if resolutionNanos is set to 0 which is currently used for testing + if (newTokens > 0 || resolutionNanos == 0 || consistentConsumedTokens || forceConsistentTokens) { // flush the pendingConsumedTokens by calling "sumThenReset" - long totalConsumedTokens = consumeTokens + pendingConsumedTokens.sumThenReset(); - // update the tokens and return the current token value - return TOKENS_UPDATER.updateAndGet(this, - currentTokens -> - // after adding new tokens, limit the tokens to the capacity - Math.min(currentTokens + newTokens, getCapacity()) - // subtract the consumed tokens - - totalConsumedTokens); + long currentPendingConsumedTokens = pendingConsumedTokens.sumThenReset(); + // calculate the token delta by subtracting the consumed tokens from the new tokens + long tokenDelta = newTokens - currentPendingConsumedTokens; + if (tokenDelta != 0 || consumeTokens != 0) { + // update the tokens and return the current token value + return TOKENS_UPDATER.updateAndGet(this, + // limit the tokens to the capacity of the bucket + currentTokens -> Math.min(currentTokens + tokenDelta, getCapacity()) + // subtract the consumed tokens from the capped tokens + - consumeTokens); + } else { + return tokens; + } } else { // eventual consistent fast path, tokens are not updated immediately @@ -187,19 +220,19 @@ private long consumeTokensAndMaybeUpdateTokensBalance(long consumeTokens, boolea * * The tokens will be updated once every resolutionNanos nanoseconds. * This method checks if the configured resolutionNanos has passed since the last update. - * If the forceUpdateTokens is true, the tokens will be updated immediately. + * If the forceConsistentTokens is true, the tokens will be updated immediately. * - * @param currentNanos the current monotonic clock time in nanoseconds - * @param forceUpdateTokens if true, the tokens will be updated immediately + * @param currentNanos the current monotonic clock time in nanoseconds + * @param forceConsistentTokens if true, the tokens are added even if the configured resolution hasn't fully passed * @return true if the tokens should be updated immediately, false otherwise */ - private boolean shouldUpdateTokensImmediately(long currentNanos, boolean forceUpdateTokens) { + private boolean shouldAddTokensImmediately(long currentNanos, boolean forceConsistentTokens) { long currentIncrement = resolutionNanos != 0 ? currentNanos / resolutionNanos : 0; long currentLastIncrement = lastIncrement; return currentIncrement == 0 || (currentIncrement > currentLastIncrement && LAST_INCREMENT_UPDATER.compareAndSet(this, currentLastIncrement, currentIncrement)) - || forceUpdateTokens; + || consistentAddedTokens || forceConsistentTokens; } /** @@ -209,10 +242,22 @@ private boolean shouldUpdateTokensImmediately(long currentNanos, boolean forceUp * @param currentNanos the current monotonic clock time in nanoseconds * @return the number of new tokens to add since the last update */ - private long calculateNewTokensSinceLastUpdate(long currentNanos) { + private long calculateNewTokensSinceLastUpdate(long currentNanos, boolean forceConsistentTokens) { + long previousLastNanos = lastNanos; + long newLastNanos; + // update lastNanos only if at least resolutionNanos/2 nanoseconds has passed since the last update + // unless consistency is needed + long minimumIncrementNanos = forceConsistentTokens || consistentAddedTokens ? 0L : resolutionNanos / 2; + if (currentNanos > previousLastNanos + minimumIncrementNanos) { + newLastNanos = currentNanos; + } else { + newLastNanos = previousLastNanos; + } long newTokens; - long previousLastNanos = LAST_NANOS_UPDATER.getAndSet(this, currentNanos); - if (previousLastNanos == 0) { + if (newLastNanos == previousLastNanos + // prevent races with a CAS update of lastNanos + || !LAST_NANOS_UPDATER.compareAndSet(this, previousLastNanos, newLastNanos) + || previousLastNanos == Long.MIN_VALUE) { newTokens = 0; } else { long durationNanos = currentNanos - previousLastNanos + REMAINDER_NANOS_UPDATER.getAndSet(this, 0); @@ -267,15 +312,14 @@ public boolean consumeTokensAndCheckIfContainsTokens(long consumeTokens) { } /** - * Returns the current token balance. When forceUpdateTokens is true, the tokens balance is updated before - * returning. If forceUpdateTokens is false, the tokens balance could be updated if the last updated happened + * Returns the current token balance. When forceConsistentTokens is true, the tokens balance is updated before + * returning. If forceConsistentTokens is false, the tokens balance could be updated if the last updated happened * more than resolutionNanos nanoseconds ago. * - * @param forceUpdateTokens if true, the tokens balance is updated before returning * @return the current token balance */ - protected long tokens(boolean forceUpdateTokens) { - long currentTokens = consumeTokensAndMaybeUpdateTokensBalance(0, forceUpdateTokens); + private long tokens() { + long currentTokens = consumeTokensAndMaybeUpdateTokensBalance(0, false); if (currentTokens != Long.MIN_VALUE) { // when currentTokens isn't Long.MIN_VALUE, the current tokens balance is known return currentTokens; @@ -295,7 +339,7 @@ public long calculateThrottlingDuration() { long currentTokens = consumeTokensAndMaybeUpdateTokensBalance(0, true); if (currentTokens == Long.MIN_VALUE) { throw new IllegalArgumentException( - "Unexpected result from updateAndConsumeTokens with forceUpdateTokens set to true"); + "Unexpected result from updateAndConsumeTokens with forceConsistentTokens set to true"); } if (currentTokens > 0) { return 0L; @@ -309,10 +353,11 @@ public long calculateThrottlingDuration() { /** * Returns the current number of tokens in the bucket. - * The token balance is updated if the configured resolutionNanos has passed since the last update. + * The token balance is updated if the configured resolutionNanos has passed since the last update unless + * consistentConsumedTokens is true. */ public final long getTokens() { - return tokens(false); + return tokens(); } public abstract long getRate(); @@ -320,25 +365,13 @@ public final long getTokens() { /** * Checks if the bucket contains tokens. * The token balance is updated before the comparison if the configured resolutionNanos has passed since the last - * update. It's possible that the returned result is not definite since the token balance is eventually consistent. + * update. It's possible that the returned result is not definite since the token balance is eventually consistent + * if consistentConsumedTokens is false. * * @return true if the bucket contains tokens, false otherwise */ public boolean containsTokens() { - return containsTokens(false); - } - - /** - * Checks if the bucket contains tokens. - * The token balance is updated before the comparison if the configured resolutionNanos has passed since the last - * update. The token balance is also updated when forceUpdateTokens is true. - * It's possible that the returned result is not definite since the token balance is eventually consistent. - * - * @param forceUpdateTokens if true, the token balance is updated before the comparison - * @return true if the bucket contains tokens, false otherwise - */ - public boolean containsTokens(boolean forceUpdateTokens) { - return tokens(forceUpdateTokens) > 0; + return tokens() > 0; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java index ee256d5a37d64..1c05f1a213e3a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/AsyncTokenBucketBuilder.java @@ -23,6 +23,8 @@ public abstract class AsyncTokenBucketBuilder> { protected MonotonicSnapshotClock clock = AsyncTokenBucket.DEFAULT_SNAPSHOT_CLOCK; protected long resolutionNanos = AsyncTokenBucket.defaultResolutionNanos; + protected boolean consistentConsumedTokens; + protected boolean consistentAddedTokens; protected AsyncTokenBucketBuilder() { } @@ -31,15 +33,47 @@ protected SELF self() { return (SELF) this; } + /** + * Set the clock source for the token bucket. It's recommended to use the {@link DefaultMonotonicSnapshotClock} + * for most use cases. + */ public SELF clock(MonotonicSnapshotClock clock) { this.clock = clock; return self(); } + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the token balance is updated, when time + * advances more than the configured resolution. This setting determines the duration of the increment. + * Setting this value to 0 will make the token balance fully consistent. There's a performance trade-off + * when setting this value to 0. + */ public SELF resolutionNanos(long resolutionNanos) { this.resolutionNanos = resolutionNanos; return self(); } + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the consumed tokens are subtracted from + * the total amount of tokens at most once during each "increment", when time advances more than the configured + * resolution. This setting determines if the consumed tokens are subtracted from tokens balance consistently. + * For high performance, it is recommended to keep this setting as false. + */ + public SELF consistentConsumedTokens(boolean consistentConsumedTokens) { + this.consistentConsumedTokens = consistentConsumedTokens; + return self(); + } + + /** + * By default, AsyncTokenBucket is eventually consistent. This means that the added tokens are calculated based + * on elapsed time at most once during each "increment", when time advances more than the configured + * resolution. This setting determines if the added tokens are calculated and added to tokens balance consistently. + * For high performance, it is recommended to keep this setting as false. + */ + public SELF consistentAddedTokens(boolean consistentAddedTokens) { + this.consistentAddedTokens = consistentAddedTokens; + return self(); + } + public abstract AsyncTokenBucket build(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java index df3843921ed55..23b9359c8042d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClock.java @@ -19,71 +19,269 @@ package org.apache.pulsar.broker.qos; +import java.util.Objects; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.LongConsumer; import java.util.function.LongSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Default implementation of {@link MonotonicSnapshotClock}. + * Default implementation of {@link MonotonicSnapshotClock} optimized for use with {@link AsyncTokenBucket}. * - * Starts a daemon thread that updates the snapshot value periodically with a configured interval. The close method - * should be called to stop the thread. + *

+ * This class provides a monotonic snapshot value that consistently increases, ensuring reliable behavior + * even in environments where the underlying clock source may not be strictly monotonic across all CPUs, + * such as certain virtualized platforms. + *

+ * + *

+ * Upon instantiation, a daemon thread is launched to periodically update the snapshot value at a configured + * interval. It is essential to invoke the {@link #close()} method to gracefully terminate this thread when it is + * no longer needed. + *

+ * + *

+ * The {@link AsyncTokenBucket} utilizes this clock to obtain tick values. It does not require a consistent value on + * every retrieval. However, when a consistent snapshot is necessary, the {@link #getTickNanos(boolean)} method + * is called with the {@code requestSnapshot} parameter set to {@code true}. + *

+ * + *

+ * By employing a single thread to update the monotonic clock value, this implementation ensures that the snapshot + * value remains strictly increasing. This approach mitigates potential inconsistencies that may arise from clock + * source discrepancies across different CPUs. + *

*/ public class DefaultMonotonicSnapshotClock implements MonotonicSnapshotClock, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(DefaultMonotonicSnapshotClock.class); - private final long sleepMillis; - private final int sleepNanos; - private final LongSupplier clockSource; - private final Thread thread; + private final TickUpdaterThread tickUpdaterThread; private volatile long snapshotTickNanos; public DefaultMonotonicSnapshotClock(long snapshotIntervalNanos, LongSupplier clockSource) { if (snapshotIntervalNanos < TimeUnit.MILLISECONDS.toNanos(1)) { throw new IllegalArgumentException("snapshotIntervalNanos must be at least 1 millisecond"); } - this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); - this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); - this.clockSource = clockSource; - updateSnapshotTickNanos(); - thread = new Thread(this::snapshotLoop, getClass().getSimpleName() + "-update-loop"); - thread.setDaemon(true); - thread.start(); + tickUpdaterThread = new TickUpdaterThread(snapshotIntervalNanos, + Objects.requireNonNull(clockSource, "clockSource must not be null"), this::setSnapshotTickNanos); + tickUpdaterThread.start(); + } + + private void setSnapshotTickNanos(long snapshotTickNanos) { + this.snapshotTickNanos = snapshotTickNanos; } /** {@inheritDoc} */ @Override public long getTickNanos(boolean requestSnapshot) { if (requestSnapshot) { - updateSnapshotTickNanos(); + tickUpdaterThread.requestUpdateAndWait(); } return snapshotTickNanos; } - private void updateSnapshotTickNanos() { - snapshotTickNanos = clockSource.getAsLong(); + @Override + public void close() { + tickUpdaterThread.interrupt(); } - private void snapshotLoop() { - try { - while (!Thread.currentThread().isInterrupted()) { - updateSnapshotTickNanos(); + /** + * A thread that updates snapshotTickNanos value periodically with a configured interval. + * The thread is started when the DefaultMonotonicSnapshotClock is created and runs until the close method is + * called. + * A single thread is used to read the clock source value since on some hardware of virtualized platforms, + * System.nanoTime() isn't strictly monotonic across all CPUs. Reading by a single thread will improve the + * stability of the read value since a single thread is scheduled on a single CPU. If the thread is migrated + * to another CPU, the clock source value might leap backward or forward, but logic in this class will handle it. + */ + private static class TickUpdaterThread extends Thread { + private final Object tickUpdateDelayMonitor = new Object(); + private final Object tickUpdatedMonitor = new Object(); + private final MonotonicLeapDetectingTickUpdater tickUpdater; + private volatile boolean running; + private boolean tickUpdateDelayMonitorNotified; + private AtomicLong requestCount = new AtomicLong(); + private final long sleepMillis; + private final int sleepNanos; + + TickUpdaterThread(long snapshotIntervalNanos, LongSupplier clockSource, LongConsumer setSnapshotTickNanos) { + super(DefaultMonotonicSnapshotClock.class.getSimpleName() + "-update-loop"); + // set as daemon thread so that it doesn't prevent the JVM from exiting + setDaemon(true); + // set the highest priority + setPriority(MAX_PRIORITY); + this.sleepMillis = TimeUnit.NANOSECONDS.toMillis(snapshotIntervalNanos); + this.sleepNanos = (int) (snapshotIntervalNanos - TimeUnit.MILLISECONDS.toNanos(sleepMillis)); + tickUpdater = new MonotonicLeapDetectingTickUpdater(clockSource, setSnapshotTickNanos, + snapshotIntervalNanos); + } + + @Override + public void run() { + try { + running = true; + long updatedForRequestCount = -1; + while (!isInterrupted()) { + try { + // track if the thread has waited for the whole duration of the snapshot interval + // before updating the tick value + boolean waitedSnapshotInterval = false; + // sleep for the configured interval on a monitor that can be notified to stop the sleep + // and update the tick value immediately. This is used in requestUpdate method. + synchronized (tickUpdateDelayMonitor) { + tickUpdateDelayMonitorNotified = false; + // only wait if no explicit request has been made since the last update + if (requestCount.get() == updatedForRequestCount) { + // if no request has been made, sleep for the configured interval + tickUpdateDelayMonitor.wait(sleepMillis, sleepNanos); + waitedSnapshotInterval = !tickUpdateDelayMonitorNotified; + } + } + updatedForRequestCount = requestCount.get(); + // update the tick value using the tick updater which will tolerate leaps backward + tickUpdater.update(waitedSnapshotInterval); + notifyAllTickUpdated(); + } catch (InterruptedException e) { + interrupt(); + break; + } + } + } catch (Throwable t) { + // report unexpected error since this would be a fatal error when the clock doesn't progress anymore + // this is very unlikely to happen, but it's better to log it in any case + LOG.error("Unexpected fatal error that stopped the clock.", t); + } finally { + LOG.info("DefaultMonotonicSnapshotClock's TickUpdaterThread stopped. {},tid={}", this, getId()); + running = false; + notifyAllTickUpdated(); + } + } + + private void notifyAllTickUpdated() { + synchronized (tickUpdatedMonitor) { + // notify all threads that are waiting for the tick value to be updated + tickUpdatedMonitor.notifyAll(); + } + } + + public void requestUpdateAndWait() { + if (!running) { + synchronized (tickUpdater) { + // thread has stopped running, fallback to update the value directly without optimizations + tickUpdater.update(false); + } + return; + } + // increment the request count that ensures that the thread will update the tick value after this request + // was made also when there's a race condition between the request and the update + // this solution doesn't prevent all races, and it's not guaranteed that the tick value is always updated + // it will prevent the request having to wait for the delayed update cycle. This is sufficient for the + // use case. + requestCount.incrementAndGet(); + synchronized (tickUpdatedMonitor) { + // notify the thread to stop waiting and update the tick value + synchronized (tickUpdateDelayMonitor) { + tickUpdateDelayMonitorNotified = true; + tickUpdateDelayMonitor.notify(); + } + // wait until the tick value has been updated try { - Thread.sleep(sleepMillis, sleepNanos); + tickUpdatedMonitor.wait(); } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - break; + currentThread().interrupt(); + } + } + } + + @Override + public synchronized void start() { + // wait until the thread is started and the tick value has been updated + synchronized (tickUpdatedMonitor) { + super.start(); + try { + tickUpdatedMonitor.wait(); + } catch (InterruptedException e) { + currentThread().interrupt(); } } - } catch (Throwable t) { - // report unexpected error since this would be a fatal error when the clock doesn't progress anymore - // this is very unlikely to happen, but it's better to log it in any case - LOG.error("Unexpected fatal error that stopped the clock.", t); } } - @Override - public void close() { - thread.interrupt(); + /** + * Handles updating the tick value in a monotonic way so that the value is always increasing, + * regardless of leaps backward in the clock source value. + */ + static class MonotonicLeapDetectingTickUpdater { + private final LongSupplier clockSource; + private final long snapshotInternalNanos; + private final long maxDeltaNanosForLeapDetection; + private final LongConsumer tickUpdatedCallback; + private long referenceClockSourceValue = Long.MIN_VALUE; + private long baseSnapshotTickNanos; + private long previousSnapshotTickNanos; + + MonotonicLeapDetectingTickUpdater(LongSupplier clockSource, LongConsumer tickUpdatedCallback, + long snapshotInternalNanos) { + this.clockSource = clockSource; + this.snapshotInternalNanos = snapshotInternalNanos; + this.maxDeltaNanosForLeapDetection = 2 * snapshotInternalNanos; + this.tickUpdatedCallback = tickUpdatedCallback; + } + + /** + * Updates the snapshot tick value. The tickUpdatedCallback is called if the value has changed. + * The value is updated in a monotonic way so that the value is always increasing, regardless of leaps backward + * in the clock source value. + * Leap detection is done by comparing the new value with the previous value and the maximum delta value. + * + * @param waitedSnapshotInterval if true, the method has waited for the snapshot interval since the previous + * call. + */ + public void update(boolean waitedSnapshotInterval) { + // get the current clock source value + long clockValue = clockSource.getAsLong(); + + // Initialization on first call + if (referenceClockSourceValue == Long.MIN_VALUE) { + referenceClockSourceValue = clockValue; + baseSnapshotTickNanos = clockValue; + previousSnapshotTickNanos = clockValue; + // update the tick value using the callback + tickUpdatedCallback.accept(clockValue); + return; + } + + // calculate the duration since the reference clock source value + // so that the snapshot value is always increasing and tolerates it when the clock source is not strictly + // monotonic across all CPUs and leaps backward + long durationSinceReference = clockValue - referenceClockSourceValue; + // calculate the new snapshot tick value as a duration since the reference clock source value + // and add it to the base snapshot tick value + long newSnapshotTickNanos = baseSnapshotTickNanos + durationSinceReference; + + // reset the reference clock source value if the clock source value leaps backward + // more than the maximum delta value + if (newSnapshotTickNanos < previousSnapshotTickNanos - maxDeltaNanosForLeapDetection) { + // when the clock source value leaps backward, reset the reference value to the new value + // for future duration calculations + referenceClockSourceValue = clockValue; + // if the updater thread has waited for the snapshot interval since the previous call, + // increment the base snapshot tick value by the snapshot interval value + long incrementWhenLeapDetected = waitedSnapshotInterval ? snapshotInternalNanos : 0; + // set the base snapshot tick value to the new value + baseSnapshotTickNanos = previousSnapshotTickNanos + incrementWhenLeapDetected; + // set the new snapshot tick value to the base value + newSnapshotTickNanos = baseSnapshotTickNanos; + } + + // update snapshotTickNanos value if the new value is greater than the previous value + if (newSnapshotTickNanos > previousSnapshotTickNanos) { + // store the previous value + previousSnapshotTickNanos = newSnapshotTickNanos; + // update the tick value using the callback + tickUpdatedCallback.accept(newSnapshotTickNanos); + } + } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java index 8edc73d1f51e3..f2eae8aed8d9c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucket.java @@ -34,15 +34,16 @@ public class DynamicRateAsyncTokenBucket extends AsyncTokenBucket { protected DynamicRateAsyncTokenBucket(double capacityFactor, LongSupplier rateFunction, MonotonicSnapshotClock clockSource, LongSupplier ratePeriodNanosFunction, - long resolutionNanos, double initialTokensFactor, + long resolutionNanos, boolean consistentConsumedTokens, + boolean consistentAddedTokens, double initialTokensFactor, double targetFillFactorAfterThrottling) { - super(clockSource, resolutionNanos); + super(clockSource, resolutionNanos, consistentConsumedTokens, consistentAddedTokens); this.capacityFactor = capacityFactor; this.rateFunction = rateFunction; this.ratePeriodNanosFunction = ratePeriodNanosFunction; this.targetFillFactorAfterThrottling = targetFillFactorAfterThrottling; this.tokens = (long) (rateFunction.getAsLong() * initialTokensFactor); - tokens(false); + getTokens(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java index 22270484c72f0..8aebecddf90c7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/DynamicRateAsyncTokenBucketBuilder.java @@ -64,9 +64,7 @@ public DynamicRateAsyncTokenBucketBuilder targetFillFactorAfterThrottling( @Override public AsyncTokenBucket build() { return new DynamicRateAsyncTokenBucket(this.capacityFactor, this.rateFunction, - this.clock, - this.ratePeriodNanosFunction, this.resolutionNanos, - this.initialFillFactor, - targetFillFactorAfterThrottling); + this.clock, this.ratePeriodNanosFunction, this.resolutionNanos, this.consistentConsumedTokens, + this.consistentAddedTokens, this.initialFillFactor, targetFillFactorAfterThrottling); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java index 627c5ee1334b2..d83290b723f07 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucket.java @@ -30,15 +30,16 @@ class FinalRateAsyncTokenBucket extends AsyncTokenBucket { private final long targetAmountOfTokensAfterThrottling; protected FinalRateAsyncTokenBucket(long capacity, long rate, MonotonicSnapshotClock clockSource, - long ratePeriodNanos, long resolutionNanos, long initialTokens) { - super(clockSource, resolutionNanos); + long ratePeriodNanos, long resolutionNanos, boolean consistentConsumedTokens, + boolean consistentAddedTokens, long initialTokens) { + super(clockSource, resolutionNanos, consistentConsumedTokens, consistentAddedTokens); this.capacity = capacity; this.rate = rate; this.ratePeriodNanos = ratePeriodNanos != -1 ? ratePeriodNanos : ONE_SECOND_NANOS; // The target amount of tokens is the amount of tokens made available in the resolution duration this.targetAmountOfTokensAfterThrottling = Math.max(this.resolutionNanos * rate / ratePeriodNanos, 1); this.tokens = initialTokens; - tokens(false); + getTokens(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java index ff4ed53c6c7fa..a292000eaa825 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/qos/FinalRateAsyncTokenBucketBuilder.java @@ -55,7 +55,7 @@ public FinalRateAsyncTokenBucketBuilder initialTokens(long initialTokens) { public AsyncTokenBucket build() { return new FinalRateAsyncTokenBucket(this.capacity != null ? this.capacity : this.rate, this.rate, this.clock, - this.ratePeriodNanos, this.resolutionNanos, + this.ratePeriodNanos, this.resolutionNanos, this.consistentConsumedTokens, this.consistentAddedTokens, this.initialTokens != null ? this.initialTokens : this.rate ); } 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 ddd436b085493..413b1b79d7a45 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 @@ -2479,6 +2479,12 @@ public long getNumberOfNamespaceBundles() { private void handleMetadataChanges(Notification n) { + if (!pulsar.isRunning()) { + // Ignore metadata changes when broker is not running + log.info("Ignoring metadata change since broker is not running (id={}, state={}) {}", pulsar.getBrokerId(), + pulsar.getState(), n); + return; + } if (n.getType() == NotificationType.Modified && NamespaceResources.pathIsFromNamespace(n.getPath())) { NamespaceName ns = NamespaceResources.namespaceFromPath(n.getPath()); handlePoliciesUpdates(ns); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java index 8255d9b6931ff..90c8de5f97a05 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PublishRateLimiterImpl.java @@ -20,11 +20,11 @@ package org.apache.pulsar.broker.service; import com.google.common.annotations.VisibleForTesting; -import io.netty.channel.EventLoopGroup; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.broker.qos.MonotonicSnapshotClock; import org.apache.pulsar.common.policies.data.Policies; @@ -32,6 +32,7 @@ import org.jctools.queues.MessagePassingQueue; import org.jctools.queues.MpscUnboundedArrayQueue; +@Slf4j public class PublishRateLimiterImpl implements PublishRateLimiter { private volatile AsyncTokenBucket tokenBucketOnMessage; private volatile AsyncTokenBucket tokenBucketOnByte; @@ -80,7 +81,7 @@ private void scheduleDecrementThrottleCount(Producer producer) { // schedule unthrottling when the throttling count is incremented to 1 // this is to avoid scheduling unthrottling multiple times for concurrent producers if (throttledProducersCount.incrementAndGet() == 1) { - EventLoopGroup executor = producer.getCnx().getBrokerService().executor(); + ScheduledExecutorService executor = producer.getCnx().getBrokerService().executor().next(); scheduleUnthrottling(executor, calculateThrottlingDurationNanos()); } } @@ -134,7 +135,11 @@ private void unthrottleQueuedProducers(ScheduledExecutorService executor) { // unthrottle as many producers as possible while there are token available while ((throttlingDuration = calculateThrottlingDurationNanos()) == 0L && (producer = unthrottlingQueue.poll()) != null) { - producer.decrementThrottleCount(); + try { + producer.decrementThrottleCount(); + } catch (Exception e) { + log.error("Failed to unthrottle producer {}", producer, e); + } throttledProducersCount.decrementAndGet(); } // if there are still producers to be unthrottled, schedule unthrottling again diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java index b29cbcd660db1..f43b134eb122a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.qos.AsyncTokenBucket; +import org.apache.pulsar.broker.qos.AsyncTokenBucketBuilder; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -76,7 +77,9 @@ public DispatchRateLimiter(BrokerService brokerService) { * @return */ public long getAvailableDispatchRateLimitOnMsg() { - return dispatchRateLimiterOnMessage == null ? -1 : Math.max(dispatchRateLimiterOnMessage.getTokens(), 0); + AsyncTokenBucket localDispatchRateLimiterOnMessage = dispatchRateLimiterOnMessage; + return localDispatchRateLimiterOnMessage == null ? -1 : + Math.max(localDispatchRateLimiterOnMessage.getTokens(), 0); } /** @@ -85,7 +88,8 @@ public long getAvailableDispatchRateLimitOnMsg() { * @return */ public long getAvailableDispatchRateLimitOnByte() { - return dispatchRateLimiterOnByte == null ? -1 : Math.max(dispatchRateLimiterOnByte.getTokens(), 0); + AsyncTokenBucket localDispatchRateLimiterOnByte = dispatchRateLimiterOnByte; + return localDispatchRateLimiterOnByte == null ? -1 : Math.max(localDispatchRateLimiterOnByte.getTokens(), 0); } /** @@ -95,11 +99,13 @@ public long getAvailableDispatchRateLimitOnByte() { * @param byteSize */ public void consumeDispatchQuota(long numberOfMessages, long byteSize) { - if (numberOfMessages > 0 && dispatchRateLimiterOnMessage != null) { - dispatchRateLimiterOnMessage.consumeTokens(numberOfMessages); + AsyncTokenBucket localDispatchRateLimiterOnMessage = dispatchRateLimiterOnMessage; + if (numberOfMessages > 0 && localDispatchRateLimiterOnMessage != null) { + localDispatchRateLimiterOnMessage.consumeTokens(numberOfMessages); } - if (byteSize > 0 && dispatchRateLimiterOnByte != null) { - dispatchRateLimiterOnByte.consumeTokens(byteSize); + AsyncTokenBucket localDispatchRateLimiterOnByte = dispatchRateLimiterOnByte; + if (byteSize > 0 && localDispatchRateLimiterOnByte != null) { + localDispatchRateLimiterOnByte.consumeTokens(byteSize); } } @@ -221,13 +227,14 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { if (msgRate > 0) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnMessage = - AsyncTokenBucket.builderForDynamicRate() + configureAsyncTokenBucket(AsyncTokenBucket.builderForDynamicRate()) .rateFunction(() -> getRelativeDispatchRateInMsg(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) .build(); } else { this.dispatchRateLimiterOnMessage = - AsyncTokenBucket.builder().rate(msgRate).ratePeriodNanos(ratePeriodNanos) + configureAsyncTokenBucket(AsyncTokenBucket.builder()) + .rate(msgRate).ratePeriodNanos(ratePeriodNanos) .build(); } } else { @@ -238,13 +245,14 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { if (byteRate > 0) { if (dispatchRate.isRelativeToPublishRate()) { this.dispatchRateLimiterOnByte = - AsyncTokenBucket.builderForDynamicRate() + configureAsyncTokenBucket(AsyncTokenBucket.builderForDynamicRate()) .rateFunction(() -> getRelativeDispatchRateInByte(dispatchRate)) .ratePeriodNanosFunction(() -> ratePeriodNanos) .build(); } else { this.dispatchRateLimiterOnByte = - AsyncTokenBucket.builder().rate(byteRate).ratePeriodNanos(ratePeriodNanos) + configureAsyncTokenBucket(AsyncTokenBucket.builder()) + .rate(byteRate).ratePeriodNanos(ratePeriodNanos) .build(); } } else { @@ -252,6 +260,11 @@ public synchronized void updateDispatchRate(DispatchRate dispatchRate) { } } + private > T configureAsyncTokenBucket(T builder) { + builder.clock(brokerService.getPulsar().getMonotonicSnapshotClock()); + return builder; + } + private long getRelativeDispatchRateInMsg(DispatchRate dispatchRate) { return (topic != null && dispatchRate != null) ? (long) topic.getLastUpdatedAvgPublishRateInMsg() + dispatchRate.getDispatchThrottlingRateInMsg() @@ -270,7 +283,8 @@ private long getRelativeDispatchRateInByte(DispatchRate dispatchRate) { * @return */ public long getDispatchRateOnMsg() { - return dispatchRateLimiterOnMessage != null ? dispatchRateLimiterOnMessage.getRate() : -1; + AsyncTokenBucket localDispatchRateLimiterOnMessage = dispatchRateLimiterOnMessage; + return localDispatchRateLimiterOnMessage != null ? localDispatchRateLimiterOnMessage.getRate() : -1; } /** @@ -279,7 +293,8 @@ public long getDispatchRateOnMsg() { * @return */ public long getDispatchRateOnByte() { - return dispatchRateLimiterOnByte != null ? dispatchRateLimiterOnByte.getRate() : -1; + AsyncTokenBucket localDispatchRateLimiterOnByte = dispatchRateLimiterOnByte; + return localDispatchRateLimiterOnByte != null ? localDispatchRateLimiterOnByte.getRate() : -1; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java index b1de10e73b76f..0f98ab94142c8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SubscribeRateLimiter.java @@ -70,7 +70,7 @@ public synchronized boolean tryAcquire(ConsumerIdentifier consumerIdentifier) { if (tokenBucket == null) { return true; } - if (!tokenBucket.containsTokens(true)) { + if (!tokenBucket.containsTokens()) { return false; } tokenBucket.consumeTokens(1); @@ -117,7 +117,11 @@ private synchronized void updateSubscribeRate(ConsumerIdentifier consumerIdentif // update subscribe-rateLimiter if (ratePerConsumer > 0) { AsyncTokenBucket tokenBucket = - AsyncTokenBucket.builder().rate(ratePerConsumer).ratePeriodNanos(ratePeriodNanos).build(); + AsyncTokenBucket.builder() + .consistentAddedTokens(true) + .consistentConsumedTokens(true) + .clock(brokerService.getPulsar().getMonotonicSnapshotClock()) + .rate(ratePerConsumer).ratePeriodNanos(ratePeriodNanos).build(); this.subscribeRateLimiter.put(consumerIdentifier, tokenBucket); } else { // subscribe-rate should be disable and close 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 8dd2fc1c3c26d..42e2c00f73acf 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 @@ -170,20 +170,26 @@ protected final void resetConfig() { protected final void internalSetup() throws Exception { init(); - lookupUrl = new URI(brokerUrl.toString()); - if (isTcpLookup) { - lookupUrl = new URI(pulsar.getBrokerServiceUrl()); - + lookupUrl = resolveLookupUrl(); + if (isTcpLookup && enableBrokerGateway) { // setup port forwarding from the advertised port to the listen port - if (enableBrokerGateway) { - InetSocketAddress gatewayAddress = new InetSocketAddress(lookupUrl.getHost(), lookupUrl.getPort()); - InetSocketAddress brokerAddress = new InetSocketAddress("127.0.0.1", pulsar.getBrokerListenPort().get()); - brokerGateway = new PortForwarder(gatewayAddress, brokerAddress); - } + InetSocketAddress gatewayAddress = new InetSocketAddress(lookupUrl.getHost(), lookupUrl.getPort()); + InetSocketAddress brokerAddress = new InetSocketAddress("127.0.0.1", pulsar.getBrokerListenPort().get()); + brokerGateway = new PortForwarder(gatewayAddress, brokerAddress); } pulsarClient = newPulsarClient(lookupUrl.toString(), 0); } + private URI resolveLookupUrl() { + if (isTcpLookup) { + return URI.create(pulsar.getBrokerServiceUrl()); + } else { + return URI.create(brokerUrl != null + ? brokerUrl.toString() + : brokerUrlTls.toString()); + } + } + protected final void internalSetup(ServiceConfiguration serviceConfiguration) throws Exception { this.conf = serviceConfiguration; internalSetup(); @@ -228,11 +234,10 @@ protected PulsarClient replacePulsarClient(ClientBuilder clientBuilder) throws P protected final void internalSetupForStatsTest() throws Exception { init(); - String lookupUrl = brokerUrl.toString(); - if (isTcpLookup) { - lookupUrl = new URI(pulsar.getBrokerServiceUrl()).toString(); + if (pulsarClient != null) { + pulsarClient.shutdown(); } - pulsarClient = newPulsarClient(lookupUrl, 1); + pulsarClient = newPulsarClient(resolveLookupUrl().toString(), 1); } protected void doInitConf() throws Exception { @@ -360,6 +365,9 @@ protected void afterPulsarStart(PulsarService pulsar) throws Exception { protected void restartBroker() throws Exception { stopBroker(); startBroker(); + if (pulsarClient == null) { + pulsarClient = newPulsarClient(lookupUrl.toString(), 0); + } } protected void stopBroker() throws Exception { @@ -384,12 +392,16 @@ protected void startBroker() throws Exception { brokerUrl = pulsar.getWebServiceAddress() != null ? new URL(pulsar.getWebServiceAddress()) : null; brokerUrlTls = pulsar.getWebServiceAddressTls() != null ? new URL(pulsar.getWebServiceAddressTls()) : null; - if (admin != null) { - admin.close(); - if (MockUtil.isMock(admin)) { - Mockito.reset(admin); + URI newLookupUrl = resolveLookupUrl(); + if (lookupUrl == null || !newLookupUrl.equals(lookupUrl)) { + lookupUrl = newLookupUrl; + if (pulsarClient != null) { + pulsarClient.shutdown(); + pulsarClient = newPulsarClient(lookupUrl.toString(), 0); } } + + closeAdmin(); PulsarAdminBuilder pulsarAdminBuilder = PulsarAdmin.builder().serviceHttpUrl(brokerUrl != null ? brokerUrl.toString() : brokerUrlTls.toString()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java index b446f9e902f2a..82793f2748d78 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/AsyncTokenBucketTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.qos; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -50,7 +51,8 @@ private void incrementMillis(long millis) { @Test void shouldAddTokensWithConfiguredRate() { asyncTokenBucket = - AsyncTokenBucket.builder().capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + AsyncTokenBucket.builder().consistentConsumedTokens(true) + .capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); incrementSeconds(5); assertEquals(asyncTokenBucket.getTokens(), 50); incrementSeconds(1); @@ -64,7 +66,7 @@ void shouldAddTokensWithConfiguredRate() { // Consume all and verify none available and then wait 1 period and check replenished asyncTokenBucket.consumeTokens(100); - assertEquals(asyncTokenBucket.tokens(true), 0); + assertEquals(asyncTokenBucket.getTokens(), 0); incrementSeconds(1); assertEquals(asyncTokenBucket.getTokens(), 10); } @@ -91,13 +93,148 @@ void shouldSupportFractionsWhenUpdatingTokens() { @Test void shouldSupportFractionsAndRetainLeftoverWhenUpdatingTokens() { asyncTokenBucket = - AsyncTokenBucket.builder().capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + AsyncTokenBucket.builder().capacity(100) + .resolutionNanos(TimeUnit.MILLISECONDS.toNanos(1)) + .rate(10) + .initialTokens(0) + .clock(clockSource) + .build(); for (int i = 0; i < 150; i++) { incrementMillis(1); } assertEquals(asyncTokenBucket.getTokens(), 1); incrementMillis(150); assertEquals(asyncTokenBucket.getTokens(), 3); + incrementMillis(1); + assertEquals(asyncTokenBucket.getTokens(), 3); + incrementMillis(99); + assertEquals(asyncTokenBucket.getTokens(), 4); + } + + @Test + void shouldSupportFractionsAndRetainLeftoverWhenUpdatingTokens2() { + asyncTokenBucket = + AsyncTokenBucket.builder().capacity(100) + .resolutionNanos(TimeUnit.MILLISECONDS.toNanos(1)) + .rate(1) + .initialTokens(0) + .clock(clockSource) + .build(); + for (int i = 0; i < 150; i++) { + incrementMillis(1); + assertEquals(asyncTokenBucket.getTokens(), 0); + } + incrementMillis(150); + assertEquals(asyncTokenBucket.getTokens(), 0); + incrementMillis(699); + assertEquals(asyncTokenBucket.getTokens(), 0); + incrementMillis(1); + assertEquals(asyncTokenBucket.getTokens(), 1); + incrementMillis(1000); + assertEquals(asyncTokenBucket.getTokens(), 2); + } + + @Test + void shouldHandleNegativeBalanceWithEventuallyConsistentTokenUpdates() { + asyncTokenBucket = + AsyncTokenBucket.builder() + // intentionally pick a coarse resolution + .resolutionNanos(TimeUnit.SECONDS.toNanos(51)) + .capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + // assert that the token balance is 0 initially + assertThat(asyncTokenBucket.getTokens()).isEqualTo(0); + + // consume tokens without exceeding the rate + for (int i = 0; i < 10000; i++) { + asyncTokenBucket.consumeTokens(500); + incrementSeconds(50); + } + + // let 9 seconds pass + incrementSeconds(9); + + // there should be 90 tokens available + assertThat(asyncTokenBucket.getTokens()).isEqualTo(90); } + @Test + void shouldNotExceedTokenBucketSizeWithNegativeTokens() { + asyncTokenBucket = + AsyncTokenBucket.builder() + // intentionally pick a coarse resolution + .resolutionNanos(TimeUnit.SECONDS.toNanos(51)) + .capacity(100).rate(10).initialTokens(0).clock(clockSource).build(); + // assert that the token balance is 0 initially + assertThat(asyncTokenBucket.getTokens()).isEqualTo(0); + + // consume tokens without exceeding the rate + for (int i = 0; i < 100; i++) { + asyncTokenBucket.consumeTokens(600); + incrementSeconds(50); + // let tokens accumulate back to 0 every 10 seconds + if ((i + 1) % 10 == 0) { + incrementSeconds(100); + } + } + + // let 9 seconds pass + incrementSeconds(9); + + // there should be 90 tokens available + assertThat(asyncTokenBucket.getTokens()).isEqualTo(90); + } + + @Test + void shouldAccuratelyCalculateTokensWhenTimeIsLaggingBehindInInconsistentUpdates() { + clockSource = requestSnapshot -> { + if (requestSnapshot) { + return manualClockSource.get(); + } else { + // let the clock lag behind + return manualClockSource.get() - TimeUnit.SECONDS.toNanos(52); + } + }; + incrementSeconds(1); + asyncTokenBucket = + AsyncTokenBucket.builder().resolutionNanos(TimeUnit.SECONDS.toNanos(51)) + .capacity(100).rate(10).initialTokens(100).clock(clockSource).build(); + assertThat(asyncTokenBucket.getTokens()).isEqualTo(100); + + // consume tokens without exceeding the rate + for (int i = 0; i < 10000; i++) { + asyncTokenBucket.consumeTokens(500); + incrementSeconds(i == 0 ? 40 : 50); + } + + // let 9 seconds pass + incrementSeconds(9); + + // there should be 90 tokens available + assertThat(asyncTokenBucket.getTokens()).isEqualTo(90); + } + + @Test + void shouldHandleEventualConsistency() { + AtomicLong offset = new AtomicLong(0); + long resolutionNanos = TimeUnit.MILLISECONDS.toNanos(1); + DefaultMonotonicSnapshotClock monotonicSnapshotClock = + new DefaultMonotonicSnapshotClock(resolutionNanos, + () -> offset.get() + manualClockSource.get()); + long initialTokens = 500L; + asyncTokenBucket = + AsyncTokenBucket.builder() + .consistentConsumedTokens(true) + .resolutionNanos(resolutionNanos) + .capacity(100000).rate(1000).initialTokens(initialTokens).clock(monotonicSnapshotClock).build(); + for (int i = 0; i < 100000; i++) { + // increment the clock by 1ms, since rate is 1000 tokens/s, this should make 1 token available + incrementMillis(1); + // consume 1 token + asyncTokenBucket.consumeTokens(1); + } + assertThat(asyncTokenBucket.getTokens()) + // since the rate is 1/ms and the test increments the clock by 1ms and consumes 1 token in each + // iteration, the tokens should be equal to the initial tokens + .isEqualTo(initialTokens); + } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java new file mode 100644 index 0000000000000..0820b439915bb --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/qos/DefaultMonotonicSnapshotClockTest.java @@ -0,0 +1,185 @@ +/* + * 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.qos; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.time.Duration; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.assertj.core.data.Offset; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +public class DefaultMonotonicSnapshotClockTest { + @DataProvider + private static Object[] booleanValues() { + return new Object[]{ true, false }; + } + + @Test(dataProvider = "booleanValues") + void testClockHandlesTimeLeapsBackwards(boolean requestSnapshot) throws InterruptedException { + long snapshotIntervalMillis = 5; + AtomicLong clockValue = new AtomicLong(1); + @Cleanup + DefaultMonotonicSnapshotClock clock = + new DefaultMonotonicSnapshotClock(Duration.ofMillis(snapshotIntervalMillis).toNanos(), + clockValue::get); + + + long previousTick = -1; + boolean leapDirection = true; + for (int i = 0; i < 10000; i++) { + clockValue.addAndGet(TimeUnit.MILLISECONDS.toNanos(1)); + long tick = clock.getTickNanos(requestSnapshot); + //log.info("i = {}, tick = {}", i, tick); + if ((i + 1) % 5 == 0) { + leapDirection = !leapDirection; + //log.info("Time leap 5 minutes backwards"); + clockValue.addAndGet(-Duration.ofMinutes(5).toNanos()); + } + if (previousTick != -1) { + assertThat(tick) + .describedAs("i = %d, tick = %d, previousTick = %d", i, tick, previousTick) + .isGreaterThanOrEqualTo(previousTick) + .isCloseTo(previousTick, + // then snapshot is requested, the time difference between the two ticks is accurate + // otherwise allow time difference at most 4 times the snapshot interval since the + // clock is updated periodically by a background thread + Offset.offset(TimeUnit.MILLISECONDS.toNanos( + requestSnapshot ? 1 : 4 * snapshotIntervalMillis))); + } + previousTick = tick; + } + } + + @Test + void testRequestUpdate() throws InterruptedException { + @Cleanup + DefaultMonotonicSnapshotClock clock = + new DefaultMonotonicSnapshotClock(Duration.ofSeconds(5).toNanos(), System::nanoTime); + long tick1 = clock.getTickNanos(false); + long tick2 = clock.getTickNanos(true); + assertThat(tick2).isGreaterThan(tick1); + } + + @Test + void testRequestingSnapshotAfterClosed() throws InterruptedException { + DefaultMonotonicSnapshotClock clock = + new DefaultMonotonicSnapshotClock(Duration.ofSeconds(5).toNanos(), System::nanoTime); + clock.close(); + long tick1 = clock.getTickNanos(true); + Thread.sleep(10); + long tick2 = clock.getTickNanos(true); + assertThat(tick2).isGreaterThan(tick1); + } + + @Test + void testConstructorValidation() { + assertThatThrownBy(() -> new DefaultMonotonicSnapshotClock(0, System::nanoTime)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("snapshotIntervalNanos must be at least 1 millisecond"); + assertThatThrownBy(() -> new DefaultMonotonicSnapshotClock(-1, System::nanoTime)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("snapshotIntervalNanos must be at least 1 millisecond"); + assertThatThrownBy(() -> new DefaultMonotonicSnapshotClock(TimeUnit.MILLISECONDS.toNanos(1), null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("clockSource must not be null"); + } + + @Test + void testFailureHandlingInClockSource() { + @Cleanup + DefaultMonotonicSnapshotClock clock = + new DefaultMonotonicSnapshotClock(Duration.ofSeconds(5).toNanos(), () -> { + throw new RuntimeException("Test clock failure"); + }); + // the exception should be propagated + assertThatThrownBy(() -> clock.getTickNanos(true)) + .isInstanceOf(RuntimeException.class) + .hasMessage("Test clock failure"); + } + + @Test + void testLeapDetectionIndependently() { + AtomicLong clockValue = new AtomicLong(0); + AtomicLong tickValue = new AtomicLong(0); + long expectedTickValue = 0; + long snapshotIntervalNanos = TimeUnit.MILLISECONDS.toNanos(1); + DefaultMonotonicSnapshotClock.MonotonicLeapDetectingTickUpdater updater = + new DefaultMonotonicSnapshotClock.MonotonicLeapDetectingTickUpdater(clockValue::get, tickValue::set, + snapshotIntervalNanos); + + updater.update(true); + + // advance the clock + clockValue.addAndGet(snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // simulate a leap backwards in time + clockValue.addAndGet(-10 * snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // advance the clock + clockValue.addAndGet(snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // simulate a leap backwards in time, without waiting a full snapshot interval + clockValue.addAndGet(-10 * snapshotIntervalNanos); + updater.update(false); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // advance the clock + clockValue.addAndGet(snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // simulate a small leap backwards in time which isn't detected, without waiting a full snapshot interval + clockValue.addAndGet(-1 * snapshotIntervalNanos); + updater.update(false); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + // clock doesn't advance for one snapshot interval + clockValue.addAndGet(snapshotIntervalNanos); + updater.update(false); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + // now the clock should advance again + clockValue.addAndGet(snapshotIntervalNanos); + expectedTickValue += snapshotIntervalNanos; + updater.update(false); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + + // simulate a leap forward + clockValue.addAndGet(10 * snapshotIntervalNanos); + // no special handling for leap forward + expectedTickValue += 10 * snapshotIntervalNanos; + updater.update(true); + assertThat(tickValue.get()).isEqualTo(expectedTickValue); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java index 392ec0d3ff46f..8343680f9bf7b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.BytesAndMessagesCount; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.ResourceGroupMonitoringClass; import org.apache.pulsar.broker.resourcegroup.ResourceGroupService.ResourceGroupUsageStatsType; @@ -58,9 +59,10 @@ @Slf4j @Test(groups = "flaky") public class RGUsageMTAggrWaitForAllMsgsTest extends ProducerConsumerBase { - @BeforeClass + @BeforeClass(alwaysRun = true) @Override protected void setup() throws Exception { + AsyncTokenBucket.switchToConsistentTokensView(); super.internalSetup(); this.prepareForOps(); @@ -91,6 +93,7 @@ public long computeLocalQuota(long confUsage, long myUsage, long[] allUsages) { @Override protected void cleanup() throws Exception { super.internalCleanup(); + AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java index 2c44ba7e23004..5c149d4e1e792 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PublishRateLimiterTest.java @@ -26,6 +26,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import io.netty.channel.EventLoop; import io.netty.channel.EventLoopGroup; import java.util.HashMap; import java.util.concurrent.TimeUnit; @@ -73,7 +74,9 @@ public void setup() throws Exception { when(transportCnx.getBrokerService()).thenReturn(brokerService); EventLoopGroup eventLoopGroup = mock(EventLoopGroup.class); when(brokerService.executor()).thenReturn(eventLoopGroup); - doReturn(null).when(eventLoopGroup).schedule(any(Runnable.class), anyLong(), any()); + EventLoop eventLoop = mock(EventLoop.class); + when(eventLoopGroup.next()).thenReturn(eventLoop); + doReturn(null).when(eventLoop).schedule(any(Runnable.class), anyLong(), any()); incrementSeconds(1); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java new file mode 100644 index 0000000000000..31c628b2bc4ca --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AbstractMessageDispatchThrottlingTest.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ScheduledExecutorService; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.pulsar.broker.qos.AsyncTokenBucket; +import org.apache.pulsar.broker.service.BrokerService; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; + +public abstract class AbstractMessageDispatchThrottlingTest extends ProducerConsumerBase { + public static T[] merge(T[] first, T[] last) { + int totalLength = first.length + last.length; + T[] result = Arrays.copyOf(first, totalLength); + int offset = first.length; + System.arraycopy(last, 0, result, offset, first.length); + return result; + } + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + AsyncTokenBucket.switchToConsistentTokensView(); + this.conf.setClusterName("test"); + internalSetup(); + producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + internalCleanup(); + AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); + } + + @AfterMethod(alwaysRun = true) + protected void reset() throws Exception { + pulsar.getConfiguration().setForceDeleteTenantAllowed(true); + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(true); + + for (String tenant : admin.tenants().getTenants()) { + for (String namespace : admin.namespaces().getNamespaces(tenant)) { + admin.namespaces().deleteNamespace(namespace, true); + } + admin.tenants().deleteTenant(tenant, true); + } + + for (String cluster : admin.clusters().getClusters()) { + admin.clusters().deleteCluster(cluster); + } + + pulsar.getConfiguration().setForceDeleteTenantAllowed(false); + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); + + producerBaseSetup(); + } + + @DataProvider(name = "subscriptions") + public Object[][] subscriptionsProvider() { + return new Object[][]{new Object[]{SubscriptionType.Shared}, {SubscriptionType.Exclusive}}; + } + + @DataProvider(name = "dispatchRateType") + public Object[][] dispatchRateProvider() { + return new Object[][]{{DispatchRateType.messageRate}, {DispatchRateType.byteRate}}; + } + + @DataProvider(name = "subscriptionAndDispatchRateType") + public Object[][] subDisTypeProvider() { + List mergeList = new LinkedList<>(); + for (Object[] sub : subscriptionsProvider()) { + for (Object[] dispatch : dispatchRateProvider()) { + mergeList.add(AbstractMessageDispatchThrottlingTest.merge(sub, dispatch)); + } + } + return mergeList.toArray(new Object[0][0]); + } + + protected void deactiveCursors(ManagedLedgerImpl ledger) throws Exception { + Field statsUpdaterField = BrokerService.class.getDeclaredField("statsUpdater"); + statsUpdaterField.setAccessible(true); + ScheduledExecutorService statsUpdater = (ScheduledExecutorService) statsUpdaterField + .get(pulsar.getBrokerService()); + statsUpdater.shutdownNow(); + ledger.getCursors().forEach(cursor -> { + ledger.deactivateCursor(cursor); + }); + } + + enum DispatchRateType { + messageRate, byteRate; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index a544c7e13bc83..5d6f0c519abc6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.spy; @@ -27,15 +28,11 @@ import static org.testng.Assert.fail; import com.google.common.collect.Sets; import java.lang.reflect.Field; -import java.util.Arrays; import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; @@ -43,7 +40,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.cache.PendingReadsManager; import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; -import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.common.policies.data.ClusterData; @@ -52,93 +49,17 @@ import org.apache.pulsar.common.policies.data.PublishRate; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; -import org.apache.pulsar.broker.qos.AsyncTokenBucket; +import org.assertj.core.data.Offset; import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -@Test(groups = "flaky") -public class MessageDispatchThrottlingTest extends ProducerConsumerBase { +@Test(groups = "broker-api") +public class MessageDispatchThrottlingTest extends AbstractMessageDispatchThrottlingTest { private static final Logger log = LoggerFactory.getLogger(MessageDispatchThrottlingTest.class); - @BeforeClass - @Override - protected void setup() throws Exception { - AsyncTokenBucket.switchToConsistentTokensView(); - this.conf.setClusterName("test"); - super.internalSetup(); - super.producerBaseSetup(); - } - - @AfterClass(alwaysRun = true) - @Override - protected void cleanup() throws Exception { - super.internalCleanup(); - AsyncTokenBucket.resetToDefaultEventualConsistentTokensView(); - } - - @AfterMethod(alwaysRun = true) - protected void reset() throws Exception { - pulsar.getConfiguration().setForceDeleteTenantAllowed(true); - pulsar.getConfiguration().setForceDeleteNamespaceAllowed(true); - - for (String tenant : admin.tenants().getTenants()) { - for (String namespace : admin.namespaces().getNamespaces(tenant)) { - admin.namespaces().deleteNamespace(namespace, true); - } - admin.tenants().deleteTenant(tenant, true); - } - - for (String cluster : admin.clusters().getClusters()) { - admin.clusters().deleteCluster(cluster); - } - - pulsar.getConfiguration().setForceDeleteTenantAllowed(false); - pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); - - super.producerBaseSetup(); - } - - - @DataProvider(name = "subscriptions") - public Object[][] subscriptionsProvider() { - return new Object[][] { new Object[] { SubscriptionType.Shared }, { SubscriptionType.Exclusive } }; - } - - @DataProvider(name = "dispatchRateType") - public Object[][] dispatchRateProvider() { - return new Object[][] { { DispatchRateType.messageRate }, { DispatchRateType.byteRate } }; - } - - @DataProvider(name = "subscriptionAndDispatchRateType") - public Object[][] subDisTypeProvider() { - List mergeList = new LinkedList<>(); - for (Object[] sub : subscriptionsProvider()) { - for (Object[] dispatch : dispatchRateProvider()) { - mergeList.add(merge(sub, dispatch)); - } - } - return mergeList.toArray(new Object[0][0]); - } - - public static T[] merge(T[] first, T[] last) { - int totalLength = first.length + last.length; - T[] result = Arrays.copyOf(first, totalLength); - int offset = first.length; - System.arraycopy(last, 0, result, offset, first.length); - return result; - } - - enum DispatchRateType { - messageRate, byteRate; - } - /** * verifies: message-rate change gets reflected immediately into topic at runtime * @@ -150,7 +71,7 @@ public void testMessageRateDynamicallyChange() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); @@ -220,7 +141,7 @@ public void testMessageRateDynamicallyChange() throws Exception { @SuppressWarnings("deprecation") @Test public void testSystemTopicDeliveryNonBlock() throws Exception { - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); final String topicName = "persistent://" + namespace + "/" + UUID.randomUUID().toString().replaceAll("-", ""); admin.topics().createNonPartitionedTopic(topicName); @@ -264,7 +185,7 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr DispatchRateType dispatchRateType) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 100; @@ -332,7 +253,7 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr public void testClusterMsgByteRateLimitingClusterConfig() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 5; final long byteRate = 1024 * 1024;// 1MB rate enough to let all msg to be delivered @@ -407,7 +328,7 @@ public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(Subscriptio throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingAll"; final int messageRate = 10; @@ -475,7 +396,7 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT conf.setDispatchThrottlingOnNonBacklogConsumerEnabled(true); log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingAll"; final String subscriptionName = "my-subscriber-name"; @@ -528,8 +449,9 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT public void testRateLimitingMultipleConsumers() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingMultipleConsumers"; + conf.setDispatchThrottlingOnNonBacklogConsumerEnabled(true); final int messageRate = 5; DispatchRate dispatchRate = DispatchRate.builder() @@ -540,7 +462,8 @@ public void testRateLimitingMultipleConsumers() throws Exception { admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); admin.namespaces().setDispatchRate(namespace, dispatchRate); // create producer and topic - Producer producer = pulsarClient.newProducer().topic(topicName).create(); + @Cleanup + Producer producer = pulsarClient.newProducer().enableBatching(false).topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); Awaitility.await() @@ -566,10 +489,15 @@ public void testRateLimitingMultipleConsumers() throws Exception { throw new RuntimeException(e); } }); + @Cleanup Consumer consumer1 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer2 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer3 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer4 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer5 = consumerBuilder.subscribe(); // deactive cursors @@ -585,15 +513,10 @@ public void testRateLimitingMultipleConsumers() throws Exception { Thread.sleep(1000); // rate limiter should have limited messages with at least 10% accuracy (or 2 messages if messageRate is low) - Assert.assertEquals(totalReceived.get(), messageRate, Math.max(messageRate / 10, 2)); + assertThat(totalReceived.get()).isCloseTo(messageRate, Offset.offset(Math.max(messageRate / 10, 2))); - consumer1.close(); - consumer2.close(); - consumer3.close(); - consumer4.close(); - consumer5.close(); - producer.close(); log.info("-- Exiting {} test --", methodName); + conf.setDispatchThrottlingOnNonBacklogConsumerEnabled(false); } @Test @@ -602,7 +525,7 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { conf.setDispatchThrottlingOnBatchMessageEnabled(true); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingMultipleConsumers"; final int messageRate = 5; @@ -614,6 +537,7 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { final int messagesPerBatch = 100; final int numProducedMessages = messageRate * messagesPerBatch; // create producer and topic + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).enableBatching(true) .batchingMaxPublishDelay(1, TimeUnit.SECONDS).batchingMaxMessages(messagesPerBatch).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); @@ -634,10 +558,15 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { log.debug("Received message [{}] in the listener", receivedMessage); totalReceived.incrementAndGet(); }); + @Cleanup Consumer consumer1 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer2 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer3 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer4 = consumerBuilder.subscribe(); + @Cleanup Consumer consumer5 = consumerBuilder.subscribe(); // deactive cursors @@ -657,12 +586,6 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { // consumer should not have received all published message due to message-rate throttling Assert.assertEquals(totalReceived.get(), numProducedMessages); - consumer1.close(); - consumer2.close(); - consumer3.close(); - consumer4.close(); - consumer5.close(); - producer.close(); log.info("-- Exiting {} test --", methodName); } @@ -670,7 +593,7 @@ public void testRateLimitingWithBatchMsgEnabled() throws Exception { public void testClusterRateLimitingConfiguration(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 5; @@ -688,12 +611,14 @@ public void testClusterRateLimitingConfiguration(SubscriptionType subscription) admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); // create producer and topic + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); int numMessages = 500; final AtomicInteger totalReceived = new AtomicInteger(0); + @Cleanup Consumer consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-subscriber-name") .subscriptionType(subscription).messageListener((c1, msg) -> { Assert.assertNotNull(msg, "Message cannot be null"); @@ -716,8 +641,6 @@ public void testClusterRateLimitingConfiguration(SubscriptionType subscription) // consumer should not have received all published message due to message-rate throttling Assert.assertNotEquals(totalReceived.get(), numMessages); - consumer.close(); - producer.close(); admin.brokers().updateDynamicConfiguration("dispatchThrottlingRatePerTopicInMsg", Integer.toString(initValue)); log.info("-- Exiting {} test --", methodName); @@ -733,7 +656,7 @@ public void testClusterRateLimitingConfiguration(SubscriptionType subscription) public void testMessageByteRateThrottlingCombined(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingAll"; final int messageRate = 5; // 5 msgs per second @@ -803,7 +726,7 @@ public void testMessageByteRateThrottlingCombined(SubscriptionType subscription) public void testGlobalNamespaceThrottling() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 5; @@ -869,7 +792,7 @@ public void testGlobalNamespaceThrottling() throws Exception { public void testNonBacklogConsumerWithThrottlingEnabled(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/throttlingBlock"; final int messageRate = 10; @@ -948,7 +871,7 @@ public void testNonBacklogConsumerWithThrottlingEnabled(SubscriptionType subscri public void testClusterPolicyOverrideConfiguration() throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName1 = "persistent://" + namespace + "/throttlingOverride1"; final String topicName2 = "persistent://" + namespace + "/throttlingOverride2"; final int clusterMessageRate = 100; @@ -1018,7 +941,7 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { public void testClosingRateLimiter(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = "persistent://" + namespace + "/closingRateLimiter" + subscription.name(); final String subName = "mySubscription" + subscription.name(); @@ -1066,7 +989,7 @@ public void testClosingRateLimiter(SubscriptionType subscription) throws Excepti @SuppressWarnings("deprecation") @Test public void testDispatchRateCompatibility2() throws Exception { - final String namespace = "my-property/dispatch-rate-compatibility"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/dispatch-rate-compatibility"); final String topicName = "persistent://" + namespace + "/t1"; final String cluster = "test"; admin.namespaces().createNamespace(namespace, Sets.newHashSet(cluster)); @@ -1112,17 +1035,6 @@ public void testDispatchRateCompatibility2() throws Exception { topic.close().get(); } - protected void deactiveCursors(ManagedLedgerImpl ledger) throws Exception { - Field statsUpdaterField = BrokerService.class.getDeclaredField("statsUpdater"); - statsUpdaterField.setAccessible(true); - ScheduledExecutorService statsUpdater = (ScheduledExecutorService) statsUpdaterField - .get(pulsar.getBrokerService()); - statsUpdater.shutdownNow(); - ledger.getCursors().forEach(cursor -> { - ledger.deactivateCursor(cursor); - }); - } - /** * It verifies that relative throttling at least dispatch messages as publish-rate. * @@ -1133,7 +1045,7 @@ protected void deactiveCursors(ManagedLedgerImpl ledger) throws Exception { public void testRelativeMessageRateLimitingThrottling(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); - final String namespace = "my-property/relative_throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/relative_throttling_ns"); final String topicName = "persistent://" + namespace + "/relative-throttle" + subscription; final int messageRate = 1; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index ce554ab2d9c00..db40ec644e9ca 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -18,8 +18,10 @@ */ package org.apache.pulsar.client.api; +import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; import com.google.common.collect.Sets; +import java.time.Duration; import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; @@ -30,6 +32,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.policies.data.DispatchRate; import org.awaitility.Awaitility; import org.slf4j.Logger; @@ -37,8 +40,8 @@ import org.testng.Assert; import org.testng.annotations.Test; -@Test(groups = "flaky") -public class SubscriptionMessageDispatchThrottlingTest extends MessageDispatchThrottlingTest { +@Test(groups = "broker-api") +public class SubscriptionMessageDispatchThrottlingTest extends AbstractMessageDispatchThrottlingTest { private static final Logger log = LoggerFactory.getLogger(SubscriptionMessageDispatchThrottlingTest.class); /** @@ -241,7 +244,7 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce admin.namespaces().setSubscriptionDispatchRate(namespace, subscriptionDispatchRate); admin.namespaces().setDispatchRate(namespace, topicDispatchRate); long initBytes = pulsar.getConfiguration().getDispatchThrottlingRatePerTopicInByte(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + brokerRate); + updateBrokerDispatchThrottlingRateInBytes(brokerRate); final int numProducedMessages = 30; final CountDownLatch latch = new CountDownLatch(numProducedMessages); @@ -272,10 +275,11 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce Assert.fail("Should only have PersistentDispatcher in this test"); } final DispatchRateLimiter subDispatchRateLimiter = subRateLimiter; - Awaitility.await().untilAsserted(() -> { + Awaitility.await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); - Assert.assertTrue(brokerDispatchRateLimiter != null - && brokerDispatchRateLimiter.getDispatchRateOnByte() > 0); + assertThat(brokerDispatchRateLimiter) + .isNotNull() + .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(brokerRate)); DispatchRateLimiter topicDispatchRateLimiter = topic.getDispatchRateLimiter().orElse(null); Assert.assertTrue(topicDispatchRateLimiter != null && topicDispatchRateLimiter.getDispatchRateOnByte() > 0); @@ -301,10 +305,7 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce consumer.close(); producer.close(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(initBytes)); - - admin.topics().delete(topicName, true); - admin.namespaces().deleteNamespace(namespace); + updateBrokerDispatchThrottlingRateInBytes(initBytes); } /** @@ -401,7 +402,7 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT private void testDispatchRate(SubscriptionType subscription, int brokerRate, int topicRate, int subRate, int expectRate) throws Exception { - final String namespace = "my-property/throttling_ns"; + final String namespace = BrokerTestUtil.newUniqueName("my-property/throttling_ns"); final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/throttlingAll"); final String subName = "my-subscriber-name-" + subscription; @@ -419,7 +420,7 @@ private void testDispatchRate(SubscriptionType subscription, admin.namespaces().setSubscriptionDispatchRate(namespace, subscriptionDispatchRate); admin.namespaces().setDispatchRate(namespace, topicDispatchRate); long initBytes = pulsar.getConfiguration().getDispatchThrottlingRatePerTopicInByte(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + brokerRate); + updateBrokerDispatchThrottlingRateInBytes(brokerRate); final int numProducedMessages = 30; final CountDownLatch latch = new CountDownLatch(numProducedMessages); @@ -450,10 +451,11 @@ private void testDispatchRate(SubscriptionType subscription, Assert.fail("Should only have PersistentDispatcher in this test"); } final DispatchRateLimiter subDispatchRateLimiter = subRateLimiter; - Awaitility.await().untilAsserted(() -> { + Awaitility.await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); - Assert.assertTrue(brokerDispatchRateLimiter != null - && brokerDispatchRateLimiter.getDispatchRateOnByte() > 0); + assertThat(brokerDispatchRateLimiter) + .isNotNull() + .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(brokerRate)); DispatchRateLimiter topicDispatchRateLimiter = topic.getDispatchRateLimiter().orElse(null); Assert.assertTrue(topicDispatchRateLimiter != null && topicDispatchRateLimiter.getDispatchRateOnByte() > 0); @@ -482,9 +484,18 @@ private void testDispatchRate(SubscriptionType subscription, consumer.close(); producer.close(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(initBytes)); - admin.topics().delete(topicName, true); - admin.namespaces().deleteNamespace(namespace); + updateBrokerDispatchThrottlingRateInBytes(initBytes); + } + + private void updateBrokerDispatchThrottlingRateInBytes(long bytes) throws PulsarAdminException { + admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(bytes)); + long expectedBytes = bytes > 0L ? bytes : -1L; + await().untilAsserted(() -> { + DispatchRateLimiter brokerDispatchRateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); + assertThat(brokerDispatchRateLimiter) + .isNotNull() + .satisfies(l -> assertThat(l.getDispatchRateOnByte()).isEqualTo(expectedBytes)); + }); } /** @@ -537,7 +548,7 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri long initBytes = pulsar.getConfiguration().getDispatchThrottlingRatePerTopicInByte(); final int byteRate = 1000; - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", "" + byteRate); + updateBrokerDispatchThrottlingRateInBytes(byteRate); Awaitility.await().untilAsserted(() -> { Assert.assertEquals(pulsar.getConfiguration().getDispatchThrottlingRateInByte(), byteRate); @@ -576,12 +587,6 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri Producer producer1 = pulsarClient.newProducer().topic(topicName1).create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2).create(); - Awaitility.await().untilAsserted(() -> { - DispatchRateLimiter rateLimiter = pulsar.getBrokerService().getBrokerDispatchRateLimiter(); - Assert.assertTrue(rateLimiter != null - && rateLimiter.getDispatchRateOnByte() > 0); - }); - long start = System.currentTimeMillis(); // Asynchronously produce messages for (int i = 0; i < numProducedMessagesEachTopic; i++) { @@ -600,7 +605,7 @@ public void testBrokerBytesRateLimitingReceiveAllMessagesAfterThrottling(Subscri consumer2.close(); producer1.close(); producer2.close(); - admin.brokers().updateDynamicConfiguration("dispatchThrottlingRateInByte", Long.toString(initBytes)); + updateBrokerDispatchThrottlingRateInBytes(initBytes); log.info("-- Exiting {} test --", methodName); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java index 1c0ae5547d53b..a848d68f37f63 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessagePublishThrottlingTest.java @@ -41,7 +41,7 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -@Test +@Test(groups = "broker-api") public class MessagePublishThrottlingTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(MessagePublishThrottlingTest.class); From d4219897e93646737d586ec0d788d3c716acaf20 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Sun, 9 Feb 2025 23:25:03 +0800 Subject: [PATCH 294/327] [improve][offload] Skip tiered-storage deployment (#23933) Signed-off-by: Zixuan Liu --- tiered-storage/pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tiered-storage/pom.xml b/tiered-storage/pom.xml index b8327f42e34a4..ad63224a698cb 100644 --- a/tiered-storage/pom.xml +++ b/tiered-storage/pom.xml @@ -39,4 +39,16 @@ jcloud file-system + + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + From 7a79c78f8e6f4b52f13be1c6441f4b007d9a00fe Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 10 Feb 2025 09:44:07 +0800 Subject: [PATCH 295/327] [fix] [client] call redeliver 1 msg but did 2 msgs (#23943) --- .../BatchMessageWithBatchIndexLevelTest.java | 62 +++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 31 ++++------ .../GrowableArrayBlockingQueue.java | 8 +++ 3 files changed, 81 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 21a843a3efc22..7fa7bf078e0c5 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 AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) 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 77a91a944ee6c..6f2ad9152d3f6 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 @@ -2831,27 +2831,18 @@ private Optional createEncryptionContext(MessageMetadata msgM private int removeExpiredMessagesFromQueue(Set messageIds) { int messagesFromQueue = 0; - Message peek = incomingMessages.peek(); - if (peek != null) { - MessageId messageId = NegativeAcksTracker.discardBatchAndPartitionIndex(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++; - MessageId id = NegativeAcksTracker.discardBatchAndPartitionIndex(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 = NegativeAcksTracker.discardBatchAndPartitionIndex(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 215b36dcc73dad91f4c9ba9a90da50540e4899a7 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 10 Feb 2025 11:19:33 +0800 Subject: [PATCH 296/327] [improve][ml] Do not switch thread to execute asyncAddEntry's core logic (#23940) --- .../mledger/impl/ManagedLedgerImpl.java | 52 ++++++++++-------- .../mledger/impl/ShadowManagedLedgerImpl.java | 16 +++--- .../ManagedLedgerInterceptorImplTest.java | 53 +++++++++++++++++++ 3 files changed, 90 insertions(+), 31 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 4f45fc67b6377..f9a0ff2620814 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 @@ -802,33 +802,41 @@ public void asyncAddEntry(ByteBuf buffer, int numberOfMessages, AddEntryCallback buffer.retain(); // Jump to specific thread to avoid contention from writers writing from different threads - executor.execute(() -> { - OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, numberOfMessages, callback, ctx, - currentLedgerTimeoutTriggered); - internalAsyncAddEntry(addOperation); - }); + final var addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, numberOfMessages, callback, ctx, + currentLedgerTimeoutTriggered); + var added = false; + try { + // Use synchronized to ensure if `addOperation` is added to queue and fails later, it will be the first + // element in `pendingAddEntries`. + synchronized (this) { + if (managedLedgerInterceptor != null) { + managedLedgerInterceptor.beforeAddEntry(addOperation, addOperation.getNumberOfMessages()); + } + final var state = STATE_UPDATER.get(this); + beforeAddEntryToQueue(state); + pendingAddEntries.add(addOperation); + added = true; + afterAddEntryToQueue(state, addOperation); + } + } catch (Throwable throwable) { + if (!added) { + addOperation.failed(ManagedLedgerException.getManagedLedgerException(throwable)); + } // else: all elements of `pendingAddEntries` will fail in another thread + } } - protected synchronized void internalAsyncAddEntry(OpAddEntry addOperation) { - if (!beforeAddEntry(addOperation)) { - return; - } - final State state = STATE_UPDATER.get(this); + protected void beforeAddEntryToQueue(State state) throws ManagedLedgerException { if (state.isFenced()) { - addOperation.failed(new ManagedLedgerFencedException()); - return; - } else if (state == State.Terminated) { - addOperation.failed(new ManagedLedgerTerminatedException("Managed ledger was already terminated")); - return; - } else if (state == State.Closed) { - addOperation.failed(new ManagedLedgerAlreadyClosedException("Managed ledger was already closed")); - return; - } else if (state == State.WriteFailed) { - addOperation.failed(new ManagedLedgerAlreadyClosedException("Waiting to recover from failure")); - return; + throw new ManagedLedgerFencedException(); } - pendingAddEntries.add(addOperation); + switch (state) { + case Terminated -> throw new ManagedLedgerTerminatedException("Managed ledger was already terminated"); + case Closed -> throw new ManagedLedgerAlreadyClosedException("Managed ledger was already closed"); + case WriteFailed -> throw new ManagedLedgerAlreadyClosedException("Waiting to recover from failure"); + } + } + protected void afterAddEntryToQueue(State state, OpAddEntry addOperation) throws ManagedLedgerException { if (state == State.ClosingLedger || state == State.CreatingLedger) { // We don't have a ready ledger to write into // We are waiting for a new ledger to be created diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java index 4b03cad8e0a1d..bae6cd66d2825 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java @@ -223,25 +223,23 @@ private void initLastConfirmedEntry() { } @Override - protected synchronized void internalAsyncAddEntry(OpAddEntry addOperation) { - if (!beforeAddEntry(addOperation)) { - return; - } + protected void beforeAddEntryToQueue(State state) throws ManagedLedgerException { if (state != State.LedgerOpened) { - addOperation.failed(new ManagedLedgerException("Managed ledger is not opened")); - return; + throw new ManagedLedgerException("Managed ledger is not opened"); } + } + @Override + protected void afterAddEntryToQueue(State state, OpAddEntry addOperation) throws ManagedLedgerException { if (addOperation.getCtx() == null || !(addOperation.getCtx() instanceof Position position)) { - addOperation.failed(new ManagedLedgerException("Illegal addOperation context object.")); - return; + pendingAddEntries.poll(); + throw new ManagedLedgerException("Illegal addOperation context object."); } if (log.isDebugEnabled()) { log.debug("[{}] Add entry into shadow ledger lh={} entries={}, pos=({},{})", name, currentLedger.getId(), currentLedgerEntries, position.getLedgerId(), position.getEntryId()); } - pendingAddEntries.add(addOperation); if (position.getLedgerId() <= currentLedger.getId()) { // Write into lastLedger if (position.getLedgerId() == currentLedger.getId()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java index b57b5ce94be42..8663019efb8c1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java @@ -21,6 +21,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -29,9 +30,12 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Random; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import lombok.Cleanup; @@ -499,4 +503,53 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { ledger.close(); } + @Test + public void testBeforeAddEntry() throws Exception { + final var interceptor = new ManagedLedgerInterceptorImpl(getBrokerEntryMetadataInterceptors(), null); + final var config = new ManagedLedgerConfig(); + final var numEntries = 100; + config.setMaxEntriesPerLedger(numEntries); + config.setManagedLedgerInterceptor(interceptor); + @Cleanup final var ml = (ManagedLedgerImpl) factory.open("test_concurrent_add_entry", config); + + final var indexesBeforeAdd = new ArrayList(); + final var batchSizes = new ArrayList(); + final var random = new Random(); + final var latch = new CountDownLatch(numEntries); + final var executor = Executors.newFixedThreadPool(3); + final var lock = new Object(); // make sure `asyncAddEntry` are called in order + for (int i = 0; i < numEntries; i++) { + final var batchSize = random.nextInt(0, 100); + final var msg = "msg-" + i; + final var callback = new AsyncCallbacks.AddEntryCallback() { + + @Override + public void addComplete(Position position, ByteBuf entryData, Object ctx) { + latch.countDown(); + } + + @Override + public void addFailed(ManagedLedgerException exception, Object ctx) { + log.error("Failed to add {}", msg, exception); + latch.countDown(); + } + }; + executor.execute(() -> { + synchronized (lock) { + batchSizes.add((long) batchSize); + indexesBeforeAdd.add(interceptor.getIndex() + 1); // index is updated in each asyncAddEntry call + ml.asyncAddEntry(Unpooled.wrappedBuffer(msg.getBytes()), batchSize, callback, null); + } + }); + } + assertTrue(latch.await(3, TimeUnit.SECONDS)); + synchronized (lock) { + for (int i = 1; i < numEntries; i++) { + final var sum = batchSizes.get(i) + batchSizes.get(i - 1); + batchSizes.set(i, sum); + } + assertEquals(indexesBeforeAdd.subList(1, numEntries), batchSizes.subList(0, numEntries - 1)); + } + executor.shutdown(); + } } From 20b3b22368b96fcfdd0aa65332b58deb4b518656 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 11 Feb 2025 02:17:53 +0800 Subject: [PATCH 297/327] [fix][ml] Fix memory leak due to duplicated RangeCache value retain operations (#23955) Co-authored-by: Lari Hotari --- .../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 cc7b3816ce14a2ddff18dbf04216769fdb8751db Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 11 Feb 2025 11:03:39 +0800 Subject: [PATCH 298/327] [fix] [ml] incorrect non-durable cursor's backlog due to concurrently trimming ledger and non-durable cursor creation (#23951) Co-authored-by: Yunze Xu --- .../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 f9a0ff2620814..7426059e576f6 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 @@ -1152,16 +1152,17 @@ public ManagedCursor newNonDurableCursor(Position startCursorPosition, String cu return cachedCursor; } - NonDurableCursorImpl cursor = new NonDurableCursorImpl(bookKeeper, this, cursorName, - 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, + startCursorPosition, initialPosition, isReadCompacted); + cursor.setActive(); + log.info("[{}] Opened new cursor: {}", name, cursor); addCursor(cursor); + return cursor; } - - return cursor; } @Override From 59cf36f217daf8ab7b2665dc6b377c0375c8e87a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Tue, 11 Feb 2025 14:25:13 +0800 Subject: [PATCH 299/327] [fix][broker] Fix seeking by timestamp can be reset the cursor position to earliest (#23919) --- .../persistent/PersistentMessageFinder.java | 19 +-- .../service/PersistentMessageFinderTest.java | 6 +- .../broker/service/SubscriptionSeekTest.java | 118 ++++++++++++++++++ 3 files changed, 123 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java index 5a4631cf205f1..e780f1672848b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service.persistent; import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.annotations.VisibleForTesting; import java.util.Optional; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import org.apache.bookkeeper.mledger.AsyncCallbacks; @@ -91,6 +92,7 @@ public void findMessages(final long timestamp, AsyncCallbacks.FindEntryCallback } } + @VisibleForTesting public static Pair getFindPositionRange(Iterable ledgerInfos, Position lastConfirmedEntry, long targetTimestamp, int ledgerCloseTimestampMaxClockSkewMillis) { @@ -105,15 +107,11 @@ public static Pair getFindPositionRange(Iterable Position start = null; Position end = null; - LedgerInfo secondToLastLedgerInfo = null; - LedgerInfo lastLedgerInfo = null; for (LedgerInfo info : ledgerInfos) { if (!info.hasTimestamp()) { // unexpected case, don't set start and end return Pair.of(null, null); } - secondToLastLedgerInfo = lastLedgerInfo; - lastLedgerInfo = info; long closeTimestamp = info.getTimestamp(); // For an open ledger, closeTimestamp is 0 if (closeTimestamp == 0) { @@ -128,19 +126,6 @@ public static Pair getFindPositionRange(Iterable break; } } - // If the second-to-last ledger's close timestamp is less than the target timestamp, then start from the - // first entry of the last ledger when there are confirmed entries in the ledger - if (lastLedgerInfo != null && secondToLastLedgerInfo != null - && secondToLastLedgerInfo.getTimestamp() > 0 - && secondToLastLedgerInfo.getTimestamp() < targetTimestampMin) { - Position firstPositionInLedger = PositionFactory.create(lastLedgerInfo.getLedgerId(), 0); - if (lastConfirmedEntry != null - && lastConfirmedEntry.compareTo(firstPositionInLedger) >= 0) { - start = firstPositionInLedger; - } else { - start = lastConfirmedEntry; - } - } return Pair.of(start, end); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java index 6f2f1f3a1a2c0..62e27eaea4169 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java @@ -636,7 +636,7 @@ public void testGetFindPositionRange_LastTimestampIsZero() { assertNotNull(range); assertNotNull(range.getLeft()); assertNull(range.getRight()); - assertEquals(range.getLeft(), PositionFactory.create(3, 0)); + assertEquals(range.getLeft(), PositionFactory.create(2, 0)); } @Test @@ -654,7 +654,7 @@ public void testGetFindPositionRange_LastTimestampIsZeroWithNoEntries() { assertNotNull(range); assertNotNull(range.getLeft()); assertNull(range.getRight()); - assertEquals(range.getLeft(), PositionFactory.create(2, 9)); + assertEquals(range.getLeft(), PositionFactory.create(2, 0)); } @Test @@ -689,7 +689,7 @@ public void testGetFindPositionRange_MixedTimestamps() { assertNotNull(range); assertNotNull(range.getLeft()); assertNotNull(range.getRight()); - assertEquals(range.getLeft(), PositionFactory.create(3, 0)); + assertEquals(range.getLeft(), PositionFactory.create(2, 0)); assertEquals(range.getRight(), PositionFactory.create(3, 9)); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java index 582d10294a5a4..3a9c5c43f1c54 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java @@ -27,8 +27,10 @@ import static org.testng.Assert.fail; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -38,6 +40,12 @@ import java.util.function.Function; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.commons.lang3.ArrayUtils; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -50,6 +58,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.ClientBuilderImpl; @@ -61,6 +70,7 @@ import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.util.RelativeTimeUtil; import org.awaitility.Awaitility; +import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -72,6 +82,10 @@ public class SubscriptionSeekTest extends BrokerTestBase { @BeforeClass @Override protected void setup() throws Exception { + conf.setManagedLedgerMinLedgerRolloverTimeMinutes(0); + conf.setManagedLedgerMaxEntriesPerLedger(10); + conf.setDefaultRetentionSizeInMB(100); + conf.setDefaultRetentionTimeInMinutes(100); super.baseSetup(); conf.setAcknowledgmentAtBatchIndexLevelEnabled(true); } @@ -489,6 +503,110 @@ public void testSeekTime() throws Exception { assertEquals(sub.getNumberOfEntriesInBacklog(false), 10); } + @Test(timeOut = 30_000) + public void testSeekByTimestamp() throws Exception { + String topicName = "persistent://prop/use/ns-abc/testSeekByTimestamp"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, "my-sub", MessageId.earliest); + + @Cleanup + Producer producer = + pulsarClient.newProducer(Schema.STRING).topic(topicName).enableBatching(false).create(); + for (int i = 0; i < 25; i++) { + producer.send(("message-" + i)); + Thread.sleep(10); + } + + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).get().get(); + + Map timestampToMessageId = new HashMap<>(); + @Cleanup + Reader reader = pulsarClient.newReader(Schema.STRING).topic(topicName).startMessageId(MessageId.earliest).create(); + while (reader.hasMessageAvailable()) { + Message message = reader.readNext(); + timestampToMessageId.put(message.getPublishTime(), message.getMessageId()); + } + + Assert.assertEquals(timestampToMessageId.size(), 25); + + PersistentSubscription subscription = topic.getSubscription("my-sub"); + ManagedCursor cursor = subscription.getCursor(); + + @Cleanup + org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName).subscriptionName("my-sub").subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe(); + long[] timestamps = timestampToMessageId.keySet().stream().mapToLong(Long::longValue).toArray(); + ArrayUtils.shuffle(timestamps); + for (long timestamp : timestamps) { + MessageIdImpl messageId = (MessageIdImpl) timestampToMessageId.get(timestamp); + consumer.seek(timestamp); + Position readPosition = cursor.getReadPosition(); + Assert.assertEquals(readPosition.getLedgerId(), messageId.getLedgerId()); + Assert.assertEquals(readPosition.getEntryId(), messageId.getEntryId()); + } + } + + @Test(timeOut = 30_000) + public void testSeekByTimestampWithLedgerTrim() throws Exception { + String topicName = "persistent://prop/use/ns-abc/testSeekByTimestampWithLedgerTrim"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, "my-sub", MessageId.earliest); + + @Cleanup + Producer producer = + pulsarClient.newProducer(Schema.STRING).topic(topicName).enableBatching(false).create(); + for (int i = 0; i < 25; i++) { + producer.send(("message-" + i)); + Thread.sleep(10); + } + + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).get().get(); + ManagedLedger ledger = topic.getManagedLedger(); + ManagedLedgerConfig config = ledger.getConfig(); + config.setRetentionTime(0, TimeUnit.SECONDS); + config.setRetentionSizeInMB(0); + + Map timestampToMessageId = new HashMap<>(); + @Cleanup + Reader reader = pulsarClient.newReader(Schema.STRING).topic(topicName).startMessageId(MessageId.earliest).create(); + while (reader.hasMessageAvailable()) { + Message message = reader.readNext(); + timestampToMessageId.put(message.getPublishTime(), message.getMessageId()); + } + + Assert.assertEquals(timestampToMessageId.size(), 25); + + PersistentSubscription subscription = topic.getSubscription("my-sub"); + ManagedCursor cursor = subscription.getCursor(); + + @Cleanup + org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName).subscriptionName("my-sub").subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe(); + long[] timestamps = timestampToMessageId.keySet().stream().mapToLong(Long::longValue).toArray(); + ArrayUtils.shuffle(timestamps); + boolean enterLedgerTrimmedBranch = false; + for (long timestamp : timestamps) { + MessageIdImpl messageId = (MessageIdImpl) timestampToMessageId.get(timestamp); + consumer.seek(timestamp); + CompletableFuture trimFuture = new CompletableFuture<>(); + ledger.trimConsumedLedgersInBackground(trimFuture); + trimFuture.get(); + Position readPosition = cursor.getReadPosition(); + Map.Entry firstLedger = ledger.getLedgersInfo().firstEntry(); + Assert.assertNotNull(firstLedger); + if (firstLedger.getKey() > messageId.getLedgerId()) { + Assert.assertEquals(readPosition.getLedgerId(), firstLedger.getKey()); + Assert.assertEquals(readPosition.getEntryId(), 0); + enterLedgerTrimmedBranch = true; + } else { + Assert.assertEquals(readPosition.getLedgerId(), messageId.getLedgerId()); + Assert.assertEquals(readPosition.getEntryId(), messageId.getEntryId()); + } + } + // May have a chance to cause flaky test, because the result of `ArrayUtils.shuffle(timestamps);` is random. + Assert.assertTrue(enterLedgerTrimmedBranch); + } + @Test public void testSeekTimeByFunction() throws Exception { final String topicName = "persistent://prop/use/ns-abc/test" + UUID.randomUUID(); From 40a3b38c48168190949d95432b5559ec19bf9e0e Mon Sep 17 00:00:00 2001 From: guan46 <48884472+guan46@users.noreply.github.com> Date: Tue, 11 Feb 2025 14:52:08 +0800 Subject: [PATCH 300/327] [improve][broker] Refactor a private method to eliminate an unnecessary parameter (#23915) --- ...tStickyKeyDispatcherMultipleConsumers.java | 33 ++++++++----------- 1 file changed, 14 insertions(+), 19 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 8bddbde02c974..9e92a2ab40dc1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -424,7 +424,7 @@ private Map> filterAndGroupEntriesForDispatching(List> filterAndGroupEntriesForDispatching(List new MutableInt(getAvailablePermits(k))); // a consumer was found for the sticky key hash and the entry can be dispatched - if (permits.intValue() > 0 - && canDispatchEntry(consumer, entry, readType, stickyKeyHash, blockedByHash)) { - // decrement the permits for the consumer - permits.decrement(); - // allow the entry to be dispatched - dispatchEntry = true; + if (permits.intValue() > 0) { + boolean canDispatchEntry = canDispatchEntry(consumer, entry, readType, stickyKeyHash); + if (canDispatchEntry) { + // decrement the permits for the consumer + permits.decrement(); + // allow the entry to be dispatched + dispatchEntry = true; + } else if (canUpdateBlockedByHash) { + blockedByHash = true; + } } } } @@ -458,7 +462,7 @@ && canDispatchEntry(consumer, entry, readType, stickyKeyHash, blockedByHash)) { // the hash is blocked, add it to the set of blocked hashes alreadyBlockedHashes.add(stickyKeyHash); } - if (blockedByHash != null && blockedByHash.isTrue()) { + if (blockedByHash) { // the entry is blocked by hash, add the consumer to the blocked set blockedByHashConsumers.add(consumer); } @@ -507,27 +511,18 @@ && canDispatchEntry(consumer, entry, readType, stickyKeyHash, blockedByHash)) { // checks if the entry can be dispatched to the consumer private boolean canDispatchEntry(Consumer consumer, Entry entry, - ReadType readType, int stickyKeyHash, - MutableBoolean blockedByHash) { + ReadType readType, int stickyKeyHash) { // If redeliveryMessages contains messages that correspond to the same hash as the entry to be dispatched // do not send those messages for order guarantee if (readType == ReadType.Normal && redeliveryMessages.containsStickyKeyHash(stickyKeyHash)) { - if (blockedByHash != null) { - blockedByHash.setTrue(); - } return false; } - if (drainingHashesRequired) { // If the hash is draining, do not send the message if (drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash)) { - if (blockedByHash != null) { - blockedByHash.setTrue(); - } return false; } } - return true; } From 367faefc2e61c68e587626d07f9d98163d3cd935 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 11 Feb 2025 17:14:18 +0200 Subject: [PATCH 301/327] [fix][ml] Fix deadlock in PendingReadsManager (#23958) --- .../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 8ca91fd34801b0bf92029c47d9978a3db27f144d Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 11 Feb 2025 12:37:09 -0800 Subject: [PATCH 302/327] [fix][sec] Upgrade to Netty 4.1.118 (#23965) --- 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 4ec8450b0c7ff..6846cf01e4965 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.2-jre 1.10.12 diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 8e45de3a98da5..6cddc05ecddf5 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -293,33 +293,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 fc97a98af6fde..7d0b9adbd3bc4 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -347,35 +347,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 ac1d68647c56d..a4cefe86a599a 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,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 69a55ca86e84f02a83d70a3eb6ccd570aceed4b3 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 11 Feb 2025 12:57:26 -0800 Subject: [PATCH 303/327] [fix] Use Alpine 3.21 in base image (#23964) --- docker/pulsar/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 121982e44285a..aec8355963958 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -17,7 +17,7 @@ # under the License. # -ARG ALPINE_VERSION=3.20 +ARG ALPINE_VERSION=3.21 ARG IMAGE_JDK_MAJOR_VERSION=21 # First create a stage with just the Pulsar tarball and scripts From 7f595cd9dc7522dd597ff58aa8b2a23737c9d841 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 11 Feb 2025 23:39:13 +0200 Subject: [PATCH 304/327] [fix][ml] Fix memory leaks in ManagedCursorInfo and ManagedLedgerInfo decompression and compression (#23960) --- .../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 6db275c3407ef7b7d308ac2eeaa3581f0d3576be Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 12 Feb 2025 07:30:02 +0800 Subject: [PATCH 305/327] [improve][broker] Avoid PersistentReplicator.expireMessages logic compute backlog twice (#23957) --- .../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 bcb1f759540b4..cc531afc6dddb 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 @@ -670,8 +670,8 @@ public void updateMessageTTL(int messageTTLInSeconds) { @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 1ab63455c1f36062cbc587244f8c0c5f71be16c5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 11 Feb 2025 17:09:51 -0800 Subject: [PATCH 306/327] [fix] Bump golang.org/x/net from 0.23.0 to 0.33.0 in /pulsar-function-go (#23861) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli --- pulsar-function-go/examples/go.mod | 2 +- pulsar-function-go/examples/go.sum | 4 ++-- pulsar-function-go/go.mod | 2 +- pulsar-function-go/go.sum | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/pulsar-function-go/examples/go.mod b/pulsar-function-go/examples/go.mod index 449ea0988bdd8..5e2f21e7d5fb8 100644 --- a/pulsar-function-go/examples/go.mod +++ b/pulsar-function-go/examples/go.mod @@ -43,7 +43,7 @@ require ( github.com/stretchr/testify v1.8.4 // indirect go.uber.org/atomic v1.7.0 // indirect golang.org/x/crypto v0.31.0 // indirect - golang.org/x/net v0.23.0 // indirect + golang.org/x/net v0.33.0 // indirect golang.org/x/oauth2 v0.13.0 // indirect golang.org/x/sys v0.28.0 // indirect golang.org/x/term v0.27.0 // indirect diff --git a/pulsar-function-go/examples/go.sum b/pulsar-function-go/examples/go.sum index e63985f12204f..35b191c221efd 100644 --- a/pulsar-function-go/examples/go.sum +++ b/pulsar-function-go/examples/go.sum @@ -473,8 +473,8 @@ golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLd golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= -golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= +golang.org/x/net v0.33.0 h1:74SYHlV8BIgHIFC/LrYkOGIwL19eTYXQ5wc6TBuO36I= +golang.org/x/net v0.33.0/go.mod h1:HXLR5J+9DxmrqMwG9qjGCxZ+zKXxBru04zlTvWlWuN4= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= diff --git a/pulsar-function-go/go.mod b/pulsar-function-go/go.mod index 3b4ac4f8a4b9e..16fd33f233f37 100644 --- a/pulsar-function-go/go.mod +++ b/pulsar-function-go/go.mod @@ -46,7 +46,7 @@ require ( github.com/spaolacci/murmur3 v1.1.0 // indirect go.uber.org/atomic v1.7.0 // indirect golang.org/x/crypto v0.31.0 // indirect - golang.org/x/net v0.23.0 // indirect + golang.org/x/net v0.33.0 // indirect golang.org/x/oauth2 v0.13.0 // indirect golang.org/x/sys v0.28.0 // indirect golang.org/x/term v0.27.0 // indirect diff --git a/pulsar-function-go/go.sum b/pulsar-function-go/go.sum index b0b3ce60fe59e..1c9fea4d11cd7 100644 --- a/pulsar-function-go/go.sum +++ b/pulsar-function-go/go.sum @@ -473,8 +473,8 @@ golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLd golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= -golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= +golang.org/x/net v0.33.0 h1:74SYHlV8BIgHIFC/LrYkOGIwL19eTYXQ5wc6TBuO36I= +golang.org/x/net v0.33.0/go.mod h1:HXLR5J+9DxmrqMwG9qjGCxZ+zKXxBru04zlTvWlWuN4= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= From 3cc22063c0077afacaf850ae1069fa51a82d327a Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 12 Feb 2025 23:06:21 +0800 Subject: [PATCH 307/327] [fix][build] Upgrade json-smart to 2.5.2 (#23966) Signed-off-by: Zixuan Liu --- pom.xml | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a4cefe86a599a..4a78139dbac3a 100644 --- a/pom.xml +++ b/pom.xml @@ -197,7 +197,7 @@ flexible messaging model and an intuitive client API. 0.4.6 2.7.5 0.4.4-hotfix1 - 2.4.10 + 2.5.2 2.16.0 8.12.1 1.9.7.Final @@ -326,6 +326,11 @@ flexible messaging model and an intuitive client API. + + net.minidev + json-smart + ${json-smart.version} + org.jline From 0a9597640ff922c00b49016af839d7c56086f82b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 12 Feb 2025 21:57:08 +0200 Subject: [PATCH 308/327] [improve][ci] Skip "OWASP dependency check" when data wasn't found in cache (#23970) --- .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 c004b61c2e362..0dfe0ce3a68a8 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -1494,15 +1494,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, and hbase 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 '!distribution/server,!distribution/io,!distribution/offloaders,!tiered-storage/file-system,!pulsar-io/flume,!pulsar-io/hbase,!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 d3ba60a0d92e80238f9478b2cec0c41fb268238d 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 309/327] [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 --- 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 03582fab49d28..7ea16bf417a82 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 b898cb9025b63b13b3bdefe16454023180ed8d5e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 13 Feb 2025 01:29:54 +0200 Subject: [PATCH 310/327] [fix][client] Fix memory leak in ClientCnx.newLookup when there's TooManyRequestsException (#23971) --- .../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 35c41455e8987..ab02c89c298ed 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 @@ -925,6 +925,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 358112767b2ebd3902ec70d9e7c7bfd6103b74bb Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 13 Feb 2025 01:33:55 +0200 Subject: [PATCH 311/327] [fix][client] Fix memory leak when message size exceeds max message size and batching is enabled (#23967) --- .../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 7262cfd11e069..403a804b605e7 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 d493e098c779d3f3102e593ef860c115065ba32c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Feb 2025 17:04:27 -0800 Subject: [PATCH 312/327] [fix] Bump org.eclipse.jetty:jetty-server from 10.0.22 to 10.0.24 in /pulsar-io/solr (#23973) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli --- pulsar-io/solr/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 7ea16bf417a82..88e09823a5b44 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -40,14 +40,14 @@ org.eclipse.jetty jetty-bom - 10.0.22 + 10.0.24 pom import org.eclipse.jetty jetty-server - 10.0.22 + 10.0.24 @@ -135,4 +135,4 @@
- \ No newline at end of file + From ee5b13af5cf229c2e4846c6d34ebda59eb82330a Mon Sep 17 00:00:00 2001 From: zhou zhuohan <843520313@qq.com> Date: Thu, 13 Feb 2025 13:29:58 +0800 Subject: [PATCH 313/327] [improve][client] Update TypedMessageBuilder deliverAfter and deliverAt api comment (#23969) --- .../org/apache/pulsar/client/api/TypedMessageBuilder.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java index 9fcbeb4d11264..08eaaad6d3784 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TypedMessageBuilder.java @@ -189,8 +189,8 @@ public interface TypedMessageBuilder extends Serializable { *

The timestamp is milliseconds and based on UTC (eg: {@link System#currentTimeMillis()}. * *

Note: messages are only delivered with delay when a consumer is consuming - * through a {@link SubscriptionType#Shared} subscription. With other subscription - * types, the messages will still be delivered immediately. + * through a {@link SubscriptionType#Shared} or {@link SubscriptionType#Key_Shared} subscription. + * With other subscription types, the messages will still be delivered immediately. * * @param timestamp * absolute timestamp indicating when the message should be delivered to consumers @@ -202,8 +202,8 @@ public interface TypedMessageBuilder extends Serializable { * Request to deliver the message only after the specified relative delay. * *

Note: messages are only delivered with delay when a consumer is consuming - * through a {@link SubscriptionType#Shared} subscription. With other subscription - * types, the messages will still be delivered immediately. + * through a {@link SubscriptionType#Shared} or {@link SubscriptionType#Key_Shared} subscription. + * With other subscription types, the messages will still be delivered immediately. * * @param delay * the amount of delay before the message will be delivered From 0e8e50ae2609e9f681bb7fb9f7d24cf118b1e6e7 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 13 Feb 2025 07:56:13 -0800 Subject: [PATCH 314/327] [fix] Potential fix for code scanning alert no. 22: HTTP response splitting (#23976) Co-authored-by: Copilot Autofix powered by AI <62310815+github-advanced-security[bot]@users.noreply.github.com> --- .../authentication/AuthenticationProviderSasl.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/pulsar-broker-auth-sasl/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderSasl.java b/pulsar-broker-auth-sasl/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderSasl.java index f8841193ba2d2..351f8d9cfd364 100644 --- a/pulsar-broker-auth-sasl/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderSasl.java +++ b/pulsar-broker-auth-sasl/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderSasl.java @@ -271,7 +271,7 @@ public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletRe } else { checkState(request.getHeader(SASL_HEADER_STATE).equalsIgnoreCase(SASL_STATE_SERVER_CHECK_TOKEN)); setResponseHeaderState(response, SASL_STATE_COMPLETE); - response.setHeader(SASL_STATE_SERVER, request.getHeader(SASL_STATE_SERVER)); + response.setHeader(SASL_STATE_SERVER, sanitizeHeaderValue(request.getHeader(SASL_STATE_SERVER))); response.setStatus(HttpServletResponse.SC_OK); if (log.isDebugEnabled()) { log.debug("[{}] Server side role token verified success: {}", request.getRequestURI(), @@ -325,4 +325,12 @@ public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletRe } } } + + private String sanitizeHeaderValue(String value) { + if (value == null) { + return null; + } + // Remove CRLF and other special characters + return value.replaceAll("[\\r\\n]", "").replaceAll("[^\\x20-\\x7E]", ""); + } } From d3ea0ee8515949808f2067c3cc2874ac379b5f28 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 14 Feb 2025 01:50:53 +0800 Subject: [PATCH 315/327] [fix][io] Fix pulsar-io:pom not found (#23979) Signed-off-by: Zixuan Liu --- 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 7ae3ee0c99ffd..031f4a2bdf5b2 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 a89299a3a74a3..38ff0fbb1bad9 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -119,6 +119,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 ed2d281b49741..d71e436d4557c 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 fcbc0823b558d..3f36a2efc4d51 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 8d3578ea2b054..9a7d8965ee425 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 7cd51431183e7..d38ed2542d117 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 7eef536b2b6b6..987c6c8cc8b75 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 fa4236bf870fe..8f7920c7255ed 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 bf21af7c0c6cc..9cc941f8b24e7 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 297ac64dd89d6..cd7ac90263dd7 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 723f79cbef0c8..b4c1423498000 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 5721ab90b6e79..5ee0ffaedb505 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 0896d079f35d7..8a168f621e2f2 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 8af034729f79e..1f62811b3a9eb 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 7d6cae61ab74f..92b9870d5f917 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 1bfc70844eb76..5f04f0e500151 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 c08ab329dbb93..6b1ec663e08dd 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 674f40f3f4a78..53093b91ee75a 100644 --- a/pulsar-io/elastic-search/pom.xml +++ b/pulsar-io/elastic-search/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/file/pom.xml b/pulsar-io/file/pom.xml index 055469a4958f4..d784d65d516b6 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 ff77be777f98d..11dc04e9a98d7 100644 --- a/pulsar-io/flume/pom.xml +++ b/pulsar-io/flume/pom.xml @@ -122,6 +122,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 2777524d2f311..32ca117da91d0 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 d20a2ef208e27..3e6bc2b45cc36 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -91,6 +91,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 41af3be06e0e6..652d6d123b394 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 4b21005a20cd6..0ef2876b8629c 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 fedafc8569578..e60e311de3cef 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 8f9d2b071c7a4..14bbc2d4bcc83 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -79,4 +79,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 4eb0370f92b48..1f9094f34c1d8 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 22e4bbcc4685f..91a74f5fdd2cd 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 83afb21a37187..310920e2a9b6d 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 5c600acf0b398..e5eaec2e9382a 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 22684a4f0e8b8..29af900ad0e57 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 a33ce25371a41..b12665eb4785a 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -214,4 +214,15 @@ + + + + org.apache.maven.plugins + maven-deploy-plugin + + ${skipDeployConnector} + + + + diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index ca64c9c194290..9ed110d4f2941 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 64edc9670a774..e2aabb3f3ca99 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 0cc3668ede5c1..b065fe1b123ba 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 3706df72e0677..0d6759a58045d 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 c882e7b9b42d0..5384948bec0f6 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 968eb81b24614..4eba7553883f1 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 a82b37ca968a2..42fbf2d1ced77 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -110,6 +110,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 0de19ce3d67a1..edb2ec19d3884 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 88e09823a5b44..4f83196aca950 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -112,6 +112,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 e909dc16d4f43..c4b38dbb10618 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 6a7284cb894f0db603fda9b4b949bc004c963df9 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 13 Feb 2025 16:09:38 -0800 Subject: [PATCH 316/327] [improve] [pip] PIP-395: Add Proxy configuration to support configurable response headers for http reverse-proxy (#23648) --- pip/pip-395.md | 81 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 81 insertions(+) create mode 100644 pip/pip-395.md diff --git a/pip/pip-395.md b/pip/pip-395.md new file mode 100644 index 0000000000000..9c5c15c83ed52 --- /dev/null +++ b/pip/pip-395.md @@ -0,0 +1,81 @@ +# PIP-395: Add Proxy configuration to support configurable response headers for http reverse-proxy + + +# Motivation + +Pulsar Proxy has a support to use it as HTTP reverse proxy to access Broker’s admin API and can also be extended to act as a reverse proxy other HTTP endpoints. Sometimes, it is very crucial to add customizable headers into the HTTP response returned by Proxy to enhance the security experience while using Proxy over HTTP. + +Response headers in a reverse HTTP proxy are critical for maintaining and enhancing the security of the web applications and services behind the proxy. These headers act as a first line of defense, hardening the web server infrastructure and protecting clients from common web vulnerabilities. + +For example, when implementing a reverse HTTP proxy, security headers such as `Referrer-Policy`, `X-Content-Type-Options`, `Strict-Transport-Security`, `X-Content-Type-Options`, etc., are useful to prevent security attacks like clickjacking, MIME-sniffing, data leakage, and more. So, such headers play a crucial role in enhancing the security posture of proxy infrastructure. + +Therefore, we would like to add support into Pulsar Proxy where users can add custom response headers by passing them into the configuration. This PIP will add this support by adding a new configuration called `proxyHttpResponseHeadersJson` where user can pass multiple headers with key-value map into the json format. Proxy server will retrieve headers from this configuration and pass it as response headers for every http request when user wants to use Pulsar Proxy as an HTTP reverse proxy. + + +# Goals + +## In Scope + +Add a new configuration `proxyHttpResponseHeadersJson` to the Proxy configuration. +eg: +``` +proxyHttpResponseHeadersJson=`{"header1":"value1","header2":"value2"}` +``` + +## Out of Scope + +# High Level Design + +# Detailed Design + +## Design & Implementation Details + +Add a new configuration `proxyHttpResponseHeadersJson` to the Proxy configuration. +This configuration will allow the user to set default headers which proxy will return into the response headers for every http request which proxy will receive as a reverse proxy. + + +### Public API +NA +### Binary protocol + +### Configuration + +### CLI + +### Metrics + +NA + +# Monitoring + +NA + +# Security Considerations + +NA + +# Backward & Forward Compatibility + +## Upgrade + +This is a new feature, and it does not affect the existing configuration. + +## Downgrade / Rollback + +Rollback will not impact the existing functionality. + +## Pulsar Geo-Replication Upgrade & Downgrade/Rollback Considerations + + + +# Alternatives + + + +# General Notes + +# Links \ No newline at end of file From 6949ad091b77a6e06d20b695b8d1673a7d310272 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 13 Feb 2025 16:10:14 -0800 Subject: [PATCH 317/327] [improve] [proxy] PIP-395: Add Proxy configuration to support configurable response headers for http reverse-proxy (#23649) --- conf/proxy.conf | 4 ++ .../proxy/server/ProxyConfiguration.java | 7 +++ .../apache/pulsar/proxy/server/WebServer.java | 51 +++++++++++++++++++ .../server/ProxyAdditionalServletTest.java | 16 +++++- 4 files changed, 76 insertions(+), 2 deletions(-) diff --git a/conf/proxy.conf b/conf/proxy.conf index 6e6c960e8009e..567cc0772a310 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -63,6 +63,10 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Default http header map to add into http-proxy for the any security requirements. +# eg: {"header1":"value"} +proxyHttpResponseHeadersJson= + # Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. webServiceHaProxyProtocolEnabled=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 b9360e403f6f4..329e6d52ba6f1 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 @@ -818,6 +818,13 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private Set proxyAdditionalServlets = new TreeSet<>(); + @FieldContext( + category = CATEGORY_PLUGIN, + doc = "Default http header map to add into http-proxy for the any security requirements " + + "eg: { \"header1\": \"val1\", \"header2\": \"val2\" }" + ) + private String proxyHttpResponseHeadersJson; + @FieldContext( category = CATEGORY_PLUGIN, doc = "List of proxy additional servlet to load, which is a list of proxy additional servlet names" diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java index 3c472135bdfb0..7591b8b54db49 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java @@ -19,6 +19,7 @@ package org.apache.pulsar.proxy.server; import static org.apache.pulsar.proxy.server.AdminProxyHandler.INIT_PARAM_REQUEST_BUFFER_SIZE; +import com.fasterxml.jackson.core.JsonProcessingException; import io.opentelemetry.api.OpenTelemetry; import io.prometheus.client.jetty.JettyStatisticsCollector; import java.io.IOException; @@ -27,12 +28,22 @@ import java.util.Arrays; import java.util.Collections; import java.util.EnumSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import javax.servlet.DispatcherType; +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletResponse; +import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.web.AuthenticationFilter; @@ -41,6 +52,7 @@ import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.common.util.PulsarSslConfiguration; import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.jetty.tls.JettySslContextFactory; @@ -242,6 +254,7 @@ private void addServlet(String basePath, ServletHolder servletHolder, ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS); context.setContextPath(basePath); context.addServlet(servletHolder, MATCH_ALL); + context.addFilter(new FilterHolder(new CustomHeaderFilter(config)), "/*", null); for (Pair attribute : attributes) { context.setAttribute(attribute.getLeft(), attribute.getRight()); } @@ -420,5 +433,43 @@ protected void refreshSslContext() { } } + static class CustomHeaderFilter implements Filter { + + Map defaultHeaders = new HashMap<>(); + + public CustomHeaderFilter(ProxyConfiguration config) { + String headerJson = config.getProxyHttpResponseHeadersJson(); + if (StringUtils.isNotBlank(headerJson)) { + try { + defaultHeaders = ObjectMapperFactory.getMapper().getObjectMapper().readerFor(Map.class) + .readValue(headerJson); + } catch (JsonProcessingException e) { + log.warn("Failed to deserialize json headers {}", headerJson, e); + } + } + } + + @Override + public void init(FilterConfig filterConfig) throws ServletException { + } + + @Override + public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) + throws IOException, ServletException { + if (defaultHeaders != null && response instanceof HttpServletResponse) { + HttpServletResponse httpResponse = (HttpServletResponse) response; + defaultHeaders.forEach((header, value) -> { + httpResponse.setHeader(header, value); + }); + + } + chain.doFilter(request, response); + } + + @Override + public void destroy() { + } + } + private static final Logger log = LoggerFactory.getLogger(WebServer.class); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java index e12224da37199..cebec2748942a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServletWithClassLoader; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlets; @@ -68,6 +69,7 @@ public class ProxyAdditionalServletTest extends MockedPulsarServiceBaseTest { private WebServer proxyWebServer; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); private Authentication proxyClientAuthentication; + private Map responseHeaders = new HashMap<>(); @Override @BeforeClass @@ -82,6 +84,9 @@ protected void setup() throws Exception { // enable full parsing feature proxyConfig.setProxyLogLevel(Optional.of(2)); proxyConfig.setClusterName(configClusterName); + responseHeaders.put("header1", "value1"); + proxyConfig.setProxyHttpResponseHeadersJson( + ObjectMapperFactory.getMapper().writer().writeValueAsString(responseHeaders)); // this is for nar package test // addServletNar(); @@ -198,11 +203,15 @@ public void test() throws IOException { int httpPort = proxyWebServer.getListenPortHTTP().get(); log.info("proxy service httpPort {}", httpPort); String paramValue = "value - " + RandomUtils.nextInt(); - String response = httpGet("http://localhost:" + httpPort + BASE_PATH + "?" + QUERY_PARAM + "=" + paramValue); + final Map headers = new HashMap<>(); + String response = httpGet("http://localhost:" + httpPort + BASE_PATH + "?" + QUERY_PARAM + "=" + paramValue, + headers); Assert.assertEquals(response, paramValue); + String headerKey = "header1"; + Assert.assertEquals(headers.get(headerKey), responseHeaders.get(headerKey)); } - String httpGet(String url) throws IOException { + String httpGet(String url, Map headers) throws IOException { OkHttpClient client = new OkHttpClient(); okhttp3.Request request = new okhttp3.Request.Builder() .get() @@ -210,6 +219,9 @@ String httpGet(String url) throws IOException { .build(); try (Response response = client.newCall(request).execute()) { + response.headers().forEach(pair -> { + headers.put(pair.getFirst(), pair.getSecond()); + }); return response.body().string(); } } From eb1391a199d2c85f1ad7ce22a4d3eefa78d2fed3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 08:56:40 +0200 Subject: [PATCH 318/327] [improve][proxy] Make keep-alive interval configurable in Pulsar Proxy (#23981) --- 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 567cc0772a310..46d84744e1297 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 329e6d52ba6f1..d89801d360b1c 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 @@ -266,6 +266,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 f8b5d0844509e..540771c86fbe4 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 937526629acf0..b645c47242546 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 d96d2cd1f6e9c..6ef24874387a8 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)); @@ -100,6 +128,11 @@ public void testProducer() throws Exception { } } + @Test + public void testKeepAliveIntervalSecondsIsConfigured() throws Exception { + assertEquals(serviceStarter.getConfig().getKeepAliveIntervalSeconds(), 25); + } + @Test public void testProduceAndConsumeMessageWithWebsocket() throws Exception { @Cleanup("stop") @@ -180,7 +213,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()); @@ -192,7 +225,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 ee8ae8d4afb3c..3e1c1200a747a 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 5443c69d84818cb4a49704f7ab7dbccf65b2179a 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 319/327] [fix][broker] Fix incorrect blockedConsumerOnUnackedMsgs value when maxUnackedMessagesPerConsumer is 1 (#23796) --- .../pulsar/broker/service/Consumer.java | 7 +++- .../broker/service/BrokerServiceTest.java | 41 +++++++++++++++++++ 2 files changed, 47 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 b46e10a20fd34..61f9d5c86b32f 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 @@ -595,6 +595,7 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map // consumer can start again consuming messages int unAckedMsgs = UNACKED_MESSAGES_UPDATER.get(ackOwnedConsumer); @@ -1090,7 +1096,6 @@ private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position) ackOwnedConsumer.blockedConsumerOnUnackedMsgs = false; flowConsumerBlockedPermits(ackOwnedConsumer); } - return true; } public PendingAcksMap 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 89727014be99e..fa76fdd5bf45c 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 @@ -1790,6 +1790,47 @@ public void testDuplicateAcknowledgement() throws Exception { .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"; From 40d5af4a43ca508ef496367925868310ea683780 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 320/327] [fix][meta] Fix ephemeral Zookeeper put which creates a persistent znode (#23984) --- .../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 4c24aa5938b93..8fd8252152898 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 @@ -439,8 +439,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 eb7a4f36da3711b9b93527d0bfe26acad1d64b1d 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 321/327] [fix][test] fix flaky testNegativeAcksWithBackoff when batch enabled. (#23986) --- .../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 7ab3e545e981e..182c952eac82d 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 @@ -256,9 +256,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 58120841ec74373b5ea718dd0325d4542dde4330 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 14 Feb 2025 08:54:59 -0800 Subject: [PATCH 322/327] [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> --- .../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 1d53d3618b9f72f8c66dad6fc2f6248d6d79d2c1 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Sat, 15 Feb 2025 07:30:56 +0800 Subject: [PATCH 323/327] [fix][broker] fix broker may lost rack information (#23331) Co-authored-by: fanjianye --- .../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 1220951ac74fb4742abbbd331d6e751234c47015 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 17 Feb 2025 10:16:49 +0800 Subject: [PATCH 324/327] [improve][client][PIP-389] Add a producer config to improve compression performance (#23525) PIP: https://github.com/apache/pulsar/pull/23526 ### Motivation The motivation of this PIP is to provide a way to improve the compression performance by skipping the compression of small messages. We want to add a new configuration compressMinMsgBodySize to the producer configuration. This configuration will allow the user to set the minimum size of the message body that will be compressed. If the message body size is less than the compressMinMsgBodySize, the message will not be compressed. --- .../impl/RawBatchMessageContainerImpl.java | 2 +- .../broker/admin/PersistentTopicsTest.java | 3 +- .../impl/ProducerConsumerInternalTest.java | 48 ++++++++++++++++- .../client/impl/ProducerMemoryLeakTest.java | 1 + .../impl/BatchMessageContainerImpl.java | 32 ++++++++--- .../pulsar/client/impl/MessageImpl.java | 5 ++ .../pulsar/client/impl/ProducerImpl.java | 54 +++++++++++-------- .../impl/conf/ProducerConfigurationData.java | 2 + 8 files changed, 115 insertions(+), 32 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchMessageContainerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchMessageContainerImpl.java index 374f1e30c0a89..9b9f79a8ec5ea 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchMessageContainerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchMessageContainerImpl.java @@ -167,7 +167,7 @@ public ByteBuf toByteBuf() { } } - ByteBuf encryptedPayload = encrypt(getCompressedBatchMetadataAndPayload()); + ByteBuf encryptedPayload = encrypt(getCompressedBatchMetadataAndPayload(false)); updateAndReserveBatchAllocatedSize(encryptedPayload.capacity()); ByteBuf metadataAndPayload = Commands.serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, encryptedPayload); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index 258c0183114fd..27f9d72413c52 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -1257,11 +1257,12 @@ public void testExamineMessageMetadata() throws Exception { admin.topics().createPartitionedTopic(topicName, 2); @Cleanup - Producer producer = pulsarClient.newProducer(Schema.STRING) + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer(Schema.STRING) .producerName("testExamineMessageMetadataProducer") .compressionType(CompressionType.LZ4) .topic(topicName + "-partition-0") .create(); + producer.getConfiguration().setCompressMinMsgBodySize(1); producer.newMessage() .keyBytes("partition123".getBytes()) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java index a06085d3d4626..4617f631207f7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java @@ -18,23 +18,26 @@ */ package org.apache.pulsar.client.impl; -import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertEquals; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; + import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.ServerCnx; import org.apache.pulsar.client.api.BatcherBuilder; +import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.api.proto.CommandCloseProducer; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; @@ -230,4 +233,47 @@ public void testRetentionPolicyByProducingMessages() throws Exception { assertEquals(internalStats.ledgers.size(), 1); }); } + + + @Test + public void testProducerCompressionMinMsgBodySize() throws PulsarClientException { + byte[] msg1022 = new byte[1022]; + byte[] msg1025 = new byte[1025]; + final String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); + @Cleanup + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer() + .topic(topicName) + .producerName("producer") + .compressionType(CompressionType.LZ4) + .create(); + @Cleanup + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName("sub") + .subscribe(); + + producer.conf.setCompressMinMsgBodySize(1024); + producer.conf.setCompressionType(CompressionType.LZ4); + // disable batch + producer.conf.setBatchingEnabled(false); + producer.newMessage().value(msg1022).send(); + MessageImpl message = (MessageImpl) consumer.receive(); + CompressionType compressionType = message.getCompressionType(); + assertEquals(compressionType, CompressionType.NONE); + producer.newMessage().value(msg1025).send(); + message = (MessageImpl) consumer.receive(); + compressionType = message.getCompressionType(); + assertEquals(compressionType, CompressionType.LZ4); + + // enable batch + producer.conf.setBatchingEnabled(true); + producer.newMessage().value(msg1022).send(); + message = (MessageImpl) consumer.receive(); + compressionType = message.getCompressionType(); + assertEquals(compressionType, CompressionType.NONE); + producer.newMessage().value(msg1025).send(); + message = (MessageImpl) consumer.receive(); + compressionType = message.getCompressionType(); + assertEquals(compressionType, CompressionType.LZ4); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLeakTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLeakTest.java index dcdfd136476c3..e366b232a639d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLeakTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLeakTest.java @@ -124,6 +124,7 @@ public void testSendMessageSizeExceeded(int maxMessageSize, CompressionType comp .compressionType(compressionType) .enableBatching(false) .create(); + producer.getConfiguration().setCompressMinMsgBodySize(1); producer.getConnectionHandler().setMaxMessageSize(maxMessageSize); MsgPayloadTouchableMessageBuilder msgBuilder = newMessage(producer); /** 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 403a804b605e7..9e0eeafc47841 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 @@ -142,6 +142,10 @@ public boolean add(MessageImpl msg, SendCallback callback) { } protected ByteBuf getCompressedBatchMetadataAndPayload() { + return getCompressedBatchMetadataAndPayload(true); + } + + protected ByteBuf getCompressedBatchMetadataAndPayload(boolean clientOperation) { int batchWriteIndex = batchedMessageMetadataAndPayload.writerIndex(); int batchReadIndex = batchedMessageMetadataAndPayload.readerIndex(); @@ -169,11 +173,23 @@ protected ByteBuf getCompressedBatchMetadataAndPayload() { } int uncompressedSize = batchedMessageMetadataAndPayload.readableBytes(); - ByteBuf compressedPayload = compressor.encode(batchedMessageMetadataAndPayload); - batchedMessageMetadataAndPayload.release(); - if (compressionType != CompressionType.NONE) { - messageMetadata.setCompression(compressionType); - messageMetadata.setUncompressedSize(uncompressedSize); + ByteBuf compressedPayload; + if (clientOperation && producer != null){ + if (compressionType != CompressionType.NONE + && uncompressedSize > producer.conf.getCompressMinMsgBodySize()) { + compressedPayload = producer.applyCompression(batchedMessageMetadataAndPayload); + messageMetadata.setCompression(compressionType); + messageMetadata.setUncompressedSize(uncompressedSize); + } else { + compressedPayload = batchedMessageMetadataAndPayload; + } + } else { + compressedPayload = compressor.encode(batchedMessageMetadataAndPayload); + batchedMessageMetadataAndPayload.release(); + if (compressionType != CompressionType.NONE) { + messageMetadata.setCompression(compressionType); + messageMetadata.setUncompressedSize(uncompressedSize); + } } // Update the current max batch size using the uncompressed size, which is what we need in any case to @@ -252,7 +268,8 @@ public OpSendMsg createOpSendMsg() throws IOException { if (messages.size() == 1) { messageMetadata.clear(); messageMetadata.copyFrom(messages.get(0).getMessageBuilder()); - ByteBuf encryptedPayload = producer.encryptMessage(messageMetadata, getCompressedBatchMetadataAndPayload()); + ByteBuf encryptedPayload = producer.encryptMessage(messageMetadata, + getCompressedBatchMetadataAndPayload()); updateAndReserveBatchAllocatedSize(encryptedPayload.capacity()); ByteBufPair cmd = producer.sendMessage(producer.producerId, messageMetadata.getSequenceId(), 1, null, messageMetadata, encryptedPayload); @@ -284,7 +301,8 @@ public OpSendMsg createOpSendMsg() throws IOException { lowestSequenceId = -1L; return op; } - ByteBuf encryptedPayload = producer.encryptMessage(messageMetadata, getCompressedBatchMetadataAndPayload()); + ByteBuf encryptedPayload = producer.encryptMessage(messageMetadata, + getCompressedBatchMetadataAndPayload()); updateAndReserveBatchAllocatedSize(encryptedPayload.capacity()); if (encryptedPayload.readableBytes() > getMaxMessageSize()) { encryptedPayload.release(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java index 72a5fd54e852b..aa98df6cda944 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java @@ -38,6 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import lombok.Getter; +import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageIdAdv; @@ -780,6 +781,10 @@ int getUncompressedSize() { return uncompressedSize; } + CompressionType getCompressionType() { + return CompressionType.valueOf(msgMetadata.getCompression().name()); + } + SchemaState getSchemaState() { return schemaState; } 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 fb2246f3a66a1..935a6251ddaf8 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 @@ -486,7 +486,8 @@ CompletableFuture internalSendWithTxnAsync(Message message, Transa * @param payload * @return a new payload */ - private ByteBuf applyCompression(ByteBuf payload) { + @VisibleForTesting + public ByteBuf applyCompression(ByteBuf payload) { ByteBuf compressedPayload = compressor.encode(payload); payload.release(); return compressedPayload; @@ -540,22 +541,29 @@ public void sendAsync(Message message, SendCallback callback) { boolean compressed = false; // Batch will be compressed when closed // If a message has a delayed delivery time, we'll always send it individually - if (!isBatchMessagingEnabled() || msgMetadata.hasDeliverAtTime()) { - compressedPayload = applyCompression(payload); - compressed = true; + if (((!isBatchMessagingEnabled() || msgMetadata.hasDeliverAtTime()))) { + if (payload.readableBytes() < conf.getCompressMinMsgBodySize()) { - // validate msg-size (For batching this will be check at the batch completion size) - int compressedSize = compressedPayload.readableBytes(); - if (compressedSize > getMaxMessageSize() && !this.conf.isChunkingEnabled()) { - compressedPayload.release(); - String compressedStr = conf.getCompressionType() != CompressionType.NONE ? "Compressed" : ""; - PulsarClientException.InvalidMessageException invalidMessageException = - new PulsarClientException.InvalidMessageException( - format("The producer %s of the topic %s sends a %s message with %d bytes that exceeds" - + " %d bytes", - producerName, topic, compressedStr, compressedSize, getMaxMessageSize())); - completeCallbackAndReleaseSemaphore(uncompressedSize, callback, invalidMessageException); - return; + } else { + compressedPayload = applyCompression(payload); + compressed = true; + + // validate msg-size (For batching this will be check at the batch completion size) + int compressedSize = compressedPayload.readableBytes(); + if (compressedSize > getMaxMessageSize() && !this.conf.isChunkingEnabled()) { + compressedPayload.release(); + String compressedStr = conf.getCompressionType() != CompressionType.NONE + ? ("compressed (" + conf.getCompressionType() + ")") + : "uncompressed"; + PulsarClientException.InvalidMessageException invalidMessageException = + new PulsarClientException.InvalidMessageException( + format("The producer %s of the topic %s sends a %s message with %d bytes that " + + "exceeds %d bytes", + producerName, topic, compressedStr, compressedSize, + getMaxMessageSize())); + completeCallbackAndReleaseSemaphore(uncompressedSize, callback, invalidMessageException); + return; + } } } @@ -577,7 +585,7 @@ public void sendAsync(Message message, SendCallback callback) { // Update the message metadata before computing the payload chunk size to avoid a large message cannot be split // into chunks. - updateMessageMetadata(msgMetadata, uncompressedSize); + updateMessageMetadata(msgMetadata, uncompressedSize, compressed); // send in chunks int totalChunks; @@ -673,7 +681,9 @@ public void sendAsync(Message message, SendCallback callback) { * @param uncompressedSize * @return the sequence id */ - private void updateMessageMetadata(final MessageMetadata msgMetadata, final int uncompressedSize) { + @SuppressWarnings("checkstyle:Indentation") + private void updateMessageMetadata(final MessageMetadata msgMetadata, final int uncompressedSize, + boolean isCompressed) { if (!msgMetadata.hasPublishTime()) { msgMetadata.setPublishTime(client.getClientClock().millis()); @@ -683,9 +693,9 @@ private void updateMessageMetadata(final MessageMetadata msgMetadata, final int // The field "uncompressedSize" is zero means the compression info were not set yet. if (msgMetadata.getUncompressedSize() <= 0) { - if (conf.getCompressionType() != CompressionType.NONE) { - msgMetadata - .setCompression(CompressionCodecProvider.convertToWireProtocol(conf.getCompressionType())); + if (conf.getCompressionType() != CompressionType.NONE && isCompressed) { + msgMetadata.setCompression( + CompressionCodecProvider.convertToWireProtocol(conf.getCompressionType())); } msgMetadata.setUncompressedSize(uncompressedSize); } @@ -777,7 +787,7 @@ private void serializeAndSendMessage(MessageImpl msg, } else { // in this case compression has not been applied by the caller // but we have to compress the payload if compression is configured - if (!compressed) { + if (!compressed && chunkPayload.readableBytes() > conf.getCompressMinMsgBodySize()) { chunkPayload = applyCompression(chunkPayload); } ByteBuf encryptedPayload = encryptMessage(msgMetadata, chunkPayload); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ProducerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ProducerConfigurationData.java index 6ec738bbf4c8d..0c770c7c9bd05 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ProducerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ProducerConfigurationData.java @@ -189,6 +189,8 @@ public class ProducerConfigurationData implements Serializable, Cloneable { ) private CompressionType compressionType = CompressionType.NONE; + private int compressMinMsgBodySize = 4 * 1024; // 4kb + // Cannot use Optional since it's not serializable private Long initialSequenceId = null; From 40b96de0410c19551d1700501e28d8a8c4a63336 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 17 Feb 2025 14:46:45 +0800 Subject: [PATCH 325/327] [fix][build] Add develops for buildtools (#23992) Signed-off-by: Zixuan Liu --- buildtools/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 6846cf01e4965..7c78924a371e2 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -35,6 +35,13 @@ jar Pulsar Build Tools + + + Apache Pulsar developers + http://pulsar.apache.org/ + + + 2024-10-14T13:32:50Z 1.8 From 5c5b44b4b238dabba38dfcd001e3f716d9a90fa4 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 17 Feb 2025 23:48:48 +0800 Subject: [PATCH 326/327] [revert] "[improve][ml] Do not switch thread to execute asyncAddEntry's core logic (#23940)" (#23994) --- .../mledger/impl/ManagedLedgerImpl.java | 52 ++++++++---------- .../mledger/impl/ShadowManagedLedgerImpl.java | 16 +++--- .../ManagedLedgerInterceptorImplTest.java | 53 ------------------- 3 files changed, 31 insertions(+), 90 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 7426059e576f6..607b6d09cc239 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 @@ -802,41 +802,33 @@ public void asyncAddEntry(ByteBuf buffer, int numberOfMessages, AddEntryCallback buffer.retain(); // Jump to specific thread to avoid contention from writers writing from different threads - final var addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, numberOfMessages, callback, ctx, - currentLedgerTimeoutTriggered); - var added = false; - try { - // Use synchronized to ensure if `addOperation` is added to queue and fails later, it will be the first - // element in `pendingAddEntries`. - synchronized (this) { - if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.beforeAddEntry(addOperation, addOperation.getNumberOfMessages()); - } - final var state = STATE_UPDATER.get(this); - beforeAddEntryToQueue(state); - pendingAddEntries.add(addOperation); - added = true; - afterAddEntryToQueue(state, addOperation); - } - } catch (Throwable throwable) { - if (!added) { - addOperation.failed(ManagedLedgerException.getManagedLedgerException(throwable)); - } // else: all elements of `pendingAddEntries` will fail in another thread - } + executor.execute(() -> { + OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, numberOfMessages, callback, ctx, + currentLedgerTimeoutTriggered); + internalAsyncAddEntry(addOperation); + }); } - protected void beforeAddEntryToQueue(State state) throws ManagedLedgerException { - if (state.isFenced()) { - throw new ManagedLedgerFencedException(); + protected synchronized void internalAsyncAddEntry(OpAddEntry addOperation) { + if (!beforeAddEntry(addOperation)) { + return; } - switch (state) { - case Terminated -> throw new ManagedLedgerTerminatedException("Managed ledger was already terminated"); - case Closed -> throw new ManagedLedgerAlreadyClosedException("Managed ledger was already closed"); - case WriteFailed -> throw new ManagedLedgerAlreadyClosedException("Waiting to recover from failure"); + final State state = STATE_UPDATER.get(this); + if (state.isFenced()) { + addOperation.failed(new ManagedLedgerFencedException()); + return; + } else if (state == State.Terminated) { + addOperation.failed(new ManagedLedgerTerminatedException("Managed ledger was already terminated")); + return; + } else if (state == State.Closed) { + addOperation.failed(new ManagedLedgerAlreadyClosedException("Managed ledger was already closed")); + return; + } else if (state == State.WriteFailed) { + addOperation.failed(new ManagedLedgerAlreadyClosedException("Waiting to recover from failure")); + return; } - } + pendingAddEntries.add(addOperation); - protected void afterAddEntryToQueue(State state, OpAddEntry addOperation) throws ManagedLedgerException { if (state == State.ClosingLedger || state == State.CreatingLedger) { // We don't have a ready ledger to write into // We are waiting for a new ledger to be created diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java index bae6cd66d2825..4b03cad8e0a1d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java @@ -223,23 +223,25 @@ private void initLastConfirmedEntry() { } @Override - protected void beforeAddEntryToQueue(State state) throws ManagedLedgerException { + protected synchronized void internalAsyncAddEntry(OpAddEntry addOperation) { + if (!beforeAddEntry(addOperation)) { + return; + } if (state != State.LedgerOpened) { - throw new ManagedLedgerException("Managed ledger is not opened"); + addOperation.failed(new ManagedLedgerException("Managed ledger is not opened")); + return; } - } - @Override - protected void afterAddEntryToQueue(State state, OpAddEntry addOperation) throws ManagedLedgerException { if (addOperation.getCtx() == null || !(addOperation.getCtx() instanceof Position position)) { - pendingAddEntries.poll(); - throw new ManagedLedgerException("Illegal addOperation context object."); + addOperation.failed(new ManagedLedgerException("Illegal addOperation context object.")); + return; } if (log.isDebugEnabled()) { log.debug("[{}] Add entry into shadow ledger lh={} entries={}, pos=({},{})", name, currentLedger.getId(), currentLedgerEntries, position.getLedgerId(), position.getEntryId()); } + pendingAddEntries.add(addOperation); if (position.getLedgerId() <= currentLedger.getId()) { // Write into lastLedger if (position.getLedgerId() == currentLedger.getId()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java index 8663019efb8c1..b57b5ce94be42 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImplTest.java @@ -21,7 +21,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -30,12 +29,9 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Random; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import lombok.Cleanup; @@ -503,53 +499,4 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { ledger.close(); } - @Test - public void testBeforeAddEntry() throws Exception { - final var interceptor = new ManagedLedgerInterceptorImpl(getBrokerEntryMetadataInterceptors(), null); - final var config = new ManagedLedgerConfig(); - final var numEntries = 100; - config.setMaxEntriesPerLedger(numEntries); - config.setManagedLedgerInterceptor(interceptor); - @Cleanup final var ml = (ManagedLedgerImpl) factory.open("test_concurrent_add_entry", config); - - final var indexesBeforeAdd = new ArrayList(); - final var batchSizes = new ArrayList(); - final var random = new Random(); - final var latch = new CountDownLatch(numEntries); - final var executor = Executors.newFixedThreadPool(3); - final var lock = new Object(); // make sure `asyncAddEntry` are called in order - for (int i = 0; i < numEntries; i++) { - final var batchSize = random.nextInt(0, 100); - final var msg = "msg-" + i; - final var callback = new AsyncCallbacks.AddEntryCallback() { - - @Override - public void addComplete(Position position, ByteBuf entryData, Object ctx) { - latch.countDown(); - } - - @Override - public void addFailed(ManagedLedgerException exception, Object ctx) { - log.error("Failed to add {}", msg, exception); - latch.countDown(); - } - }; - executor.execute(() -> { - synchronized (lock) { - batchSizes.add((long) batchSize); - indexesBeforeAdd.add(interceptor.getIndex() + 1); // index is updated in each asyncAddEntry call - ml.asyncAddEntry(Unpooled.wrappedBuffer(msg.getBytes()), batchSize, callback, null); - } - }); - } - assertTrue(latch.await(3, TimeUnit.SECONDS)); - synchronized (lock) { - for (int i = 1; i < numEntries; i++) { - final var sum = batchSizes.get(i) + batchSizes.get(i - 1); - batchSizes.set(i, sum); - } - assertEquals(indexesBeforeAdd.subList(1, numEntries), batchSizes.subList(0, numEntries - 1)); - } - executor.shutdown(); - } } From df5197212e8806c7d1907dedfcdfd9e40a4f0ea5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 18:54:43 +0200 Subject: [PATCH 327/327] [fix][meta] Fix ephemeral handling of ZK nodes and fix MockZooKeeper ephemeral and ZK stat handling (#23988) --- .../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 | 128 +- .../apache/pulsar/metadata/CounterTest.java | 3 +- .../pulsar/metadata/MetadataCacheTest.java | 29 +- .../metadata/MetadataStoreExtendedTest.java | 6 +- .../pulsar/metadata/MetadataStoreTest.java | 27 +- .../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, 1199 insertions(+), 761 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 7ca2b926db7db..591b061253d3b 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 @@ -285,7 +285,7 @@ public static CompletableFuture> applyNamespacePoliciesAsync( 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); } @@ -293,7 +293,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 48a6121b9dd13..f86b608d93722 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 @@ -583,7 +583,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 2e198eb99752e..f191fc3a38423 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 @@ -1012,7 +1012,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 3ee365cdd4571..27c73edc6b597 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 42e2c00f73acf..5be5c7544524f 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 @@ -150,6 +150,9 @@ public static String getTlsFileForClient(String name) { private final List closeables = new ArrayList<>(); + // Set to true in test's constructor to use a real Zookeeper (TestZKServer) + protected boolean useTestZookeeper; + public MockedPulsarServiceBaseTest() { resetConfig(); } @@ -363,7 +366,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(); if (pulsarClient == null) { pulsarClient = newPulsarClient(lookupUrl.toString(), 0); @@ -461,7 +471,6 @@ protected PulsarTestContext.Builder createPulsarTestContextBuilder(ServiceConfig PulsarTestContext.Builder builder = PulsarTestContext.builder() .spyByDefault() .config(conf) - .withMockZookeeper(true) .pulsarServiceCustomizer(pulsarService -> { try { beforePulsarStart(pulsarService); @@ -470,9 +479,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); + } + } + protected PulsarTestContext createAdditionalPulsarTestContext(ServiceConfiguration conf) throws Exception { return createAdditionalPulsarTestContext(conf, null); } 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 6403c3bcec4c3..f8205a2e939a3 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,7 +19,6 @@ package org.apache.pulsar.broker.testcontext; -import com.google.common.util.concurrent.MoreExecutors; import io.netty.channel.EventLoopGroup; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; @@ -27,7 +26,6 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.time.Duration; -import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Optional; @@ -37,6 +35,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; @@ -65,6 +64,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; @@ -72,9 +72,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; @@ -161,6 +163,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; @@ -409,6 +415,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 @@ -476,17 +487,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; } /** @@ -676,6 +726,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", @@ -720,6 +784,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 07deb9007c487..d0cb4266ae10a 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 @@ -92,6 +92,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; @@ -112,15 +113,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); @@ -129,10 +156,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; } @Override @@ -295,9 +355,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(); @@ -328,6 +390,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(); } /** @@ -375,12 +439,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); @@ -417,10 +475,6 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { consumer.acknowledgeCumulative(msg); consumer.close(); producer.close(); - - // disable authorization - pulsar.getConfiguration().setAuthorizationEnabled(false); - loadManager2 = null; } /** @@ -538,18 +592,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(); @@ -753,11 +795,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(); @@ -875,12 +912,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 = @@ -1033,12 +1064,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(); @@ -1197,6 +1222,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; @@ -1312,7 +1340,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 0cf2e49d35bee..01d2f107dffe0 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 8fd8252152898..dab3553f8ce1b 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; @@ -129,12 +130,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); @@ -478,7 +484,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 ccbdb8cef64c5..9724d2d6ff65d 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 @@ -212,6 +212,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); @@ -256,6 +257,9 @@ private static void callCloseables(List closeables) { 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 d0265e3ca44ee..03a6b91b62d4b 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 @@ -25,11 +25,18 @@ import io.streamnative.oxia.testcontainers.OxiaContainer; 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; @@ -37,21 +44,47 @@ 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; - protected OxiaContainer oxiaServer; + 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(); @@ -67,40 +100,72 @@ public void cleanup() throws Exception { oxiaServer.close(); oxiaServer = 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())}, {"Oxia", stringSupplier(() -> "oxia://" + getOxiaServerConnectString())}, + {"MockZooKeeper", stringSupplier(() -> mockZkUrl)}, }; } @DataProvider(name = "distributedImpl") public Object[][] distributedImplementations() { - return new Object[][]{ - {"ZooKeeper", stringSupplier(() -> zks.getConnectionString())}, - {"Etcd", stringSupplier(() -> "etcd:" + getEtcdClusterConnectString())}, - {"Oxia", stringSupplier(() -> "oxia://" + getOxiaServerConnectString())}, - }; + return filterImplementations("ZooKeeper", "Etcd", "Oxia"); + } + + @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); } protected synchronized String getOxiaServerConnectString() { + if (!running) { + return null; + } if (oxiaServer == null) { oxiaServer = new OxiaContainer(OxiaContainer.DEFAULT_IMAGE_NAME); oxiaServer.start(); @@ -109,6 +174,9 @@ protected synchronized String getOxiaServerConnectString() { } private synchronized String getEtcdClusterConnectString() { + if (!running) { + return null; + } if (etcdCluster == null) { etcdCluster = EtcdClusterExtension.builder().withClusterName("test").withNodes(1).withSsl(false).build() .cluster(); @@ -118,7 +186,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() { @@ -164,4 +251,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 ddd975e422ab8..d3f2995a84505 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 @@ -71,7 +71,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 @@ -113,14 +112,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()); @@ -185,7 +177,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 @@ -495,11 +487,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); @@ -513,7 +504,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))); @@ -524,11 +516,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 2c589dfd48222..a7b1dcf6bf02b 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; @@ -68,6 +69,7 @@ import org.assertj.core.util.Lists; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; +import org.testng.SkipException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -415,8 +417,8 @@ public void testDeleteRecursive(String provider, Supplier urlSupplier) t @Test(dataProvider = "impl") public void testDeleteUnusedDirectories(String provider, Supplier urlSupplier) throws Exception { - if (provider.equals("Oxia")) { - return; + if (provider.equals("Oxia") || provider.equals("MockZooKeeper")) { + throw new SkipException("Oxia and MockZooKeeper do not support deleteUnusedDirectories"); } @Cleanup @@ -432,18 +434,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()); } @@ -549,6 +551,7 @@ public void testOxiaLoadConfigFromFile() throws Exception { builder.configFilePath("src/test/resources/oxia_client.conf"); MetadataStoreConfig config = builder.build(); + @Cleanup OxiaMetadataStore store = (OxiaMetadataStore) MetadataStoreFactory.create(oxia, config); var client = (AsyncOxiaClient) WhiteboxImpl.getInternalState(store, "client"); var sessionManager = (SessionManager) WhiteboxImpl.getInternalState(client, "sessionManager"); @@ -666,21 +669,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 f8a51602686ed..f4bac7bb622aa 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 = @@ -407,14 +407,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()); @@ -426,18 +428,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) { @@ -449,15 +455,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 6ede02b67136e..a0f726ed3dc9d 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 @@ -21,6 +21,10 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import io.opentelemetry.api.OpenTelemetry; +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; @@ -32,26 +36,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(); } }